Merge branch 'master' of github.com:ClickHouse/ClickHouse into select_constants

This commit is contained in:
Guillaume Tassery 2020-02-04 05:09:12 +01:00
commit a79a9cf078
58 changed files with 803 additions and 49 deletions

View File

@ -203,6 +203,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \ M(SettingUInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \
M(SettingString, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(SettingString, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \
M(SettingUInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(SettingUInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \
M(SettingBool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \
\ \
M(SettingBool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \ M(SettingBool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \
\ \
@ -338,7 +339,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \
M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \ M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \
M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \ M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \
M(SettingBool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line will be \\r\\n instead of \\n.", 0) \ M(SettingBool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \
M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \ M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \
\ \
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \

View File

@ -97,6 +97,7 @@ static FormatSettings getOutputFormatSetting(const Settings & settings, const Co
format_settings.template_settings.resultset_format = settings.format_template_resultset; format_settings.template_settings.resultset_format = settings.format_template_resultset;
format_settings.template_settings.row_format = settings.format_template_row; format_settings.template_settings.row_format = settings.format_template_row;
format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter;
format_settings.tsv.crlf_end_of_line = settings.output_format_tsv_crlf_end_of_line;
format_settings.write_statistics = settings.output_format_write_statistics; format_settings.write_statistics = settings.output_format_write_statistics;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
format_settings.schema.format_schema = settings.format_schema; format_settings.schema.format_schema = settings.format_schema;

View File

@ -64,6 +64,7 @@ struct FormatSettings
struct TSV struct TSV
{ {
bool empty_as_default = false; bool empty_as_default = false;
bool crlf_end_of_line = false;
}; };
TSV tsv; TSV tsv;

View File

@ -0,0 +1,66 @@
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/NullWriteBuffer.h>
namespace DB
{
/// Returns size on disk for *block* (without taking into account compression).
class FunctionBlockSerializedSize : public IFunction
{
public:
static constexpr auto name = "blockSerializedSize";
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionBlockSerializedSize>();
}
String getName() const override { return name; }
bool useDefaultImplementationForNulls() const override { return false; }
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{
return std::make_shared<DataTypeUInt64>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
UInt64 size = 0;
for (size_t i = 0; i < arguments.size(); ++i)
size += blockSerializedSizeOne(block.getByPosition(arguments[i]));
block.getByPosition(result).column = DataTypeUInt64().createColumnConst(
input_rows_count, size)->convertToFullColumnIfConst();
}
UInt64 blockSerializedSizeOne(const ColumnWithTypeAndName & elem) const
{
ColumnPtr full_column = elem.column->convertToFullColumnIfConst();
IDataType::SerializeBinaryBulkSettings settings;
NullWriteBuffer out;
settings.getter = [&out](IDataType::SubstreamPath) -> WriteBuffer * { return &out; };
IDataType::SerializeBinaryBulkStatePtr state;
elem.type->serializeBinaryBulkWithMultipleStreams(*full_column,
0 /** offset */, 0 /** limit */,
settings, state);
return out.count();
}
};
void registerFunctionBlockSerializedSize(FunctionFactory & factory)
{
factory.registerFunction<FunctionBlockSerializedSize>();
}
}

View File

@ -14,6 +14,7 @@ void registerFunctionFQDN(FunctionFactory &);
void registerFunctionVisibleWidth(FunctionFactory &); void registerFunctionVisibleWidth(FunctionFactory &);
void registerFunctionToTypeName(FunctionFactory &); void registerFunctionToTypeName(FunctionFactory &);
void registerFunctionGetSizeOfEnumType(FunctionFactory &); void registerFunctionGetSizeOfEnumType(FunctionFactory &);
void registerFunctionBlockSerializedSize(FunctionFactory &);
void registerFunctionToColumnTypeName(FunctionFactory &); void registerFunctionToColumnTypeName(FunctionFactory &);
void registerFunctionDumpColumnStructure(FunctionFactory &); void registerFunctionDumpColumnStructure(FunctionFactory &);
void registerFunctionDefaultValueOfArgumentType(FunctionFactory &); void registerFunctionDefaultValueOfArgumentType(FunctionFactory &);
@ -72,6 +73,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
registerFunctionVisibleWidth(factory); registerFunctionVisibleWidth(factory);
registerFunctionToTypeName(factory); registerFunctionToTypeName(factory);
registerFunctionGetSizeOfEnumType(factory); registerFunctionGetSizeOfEnumType(factory);
registerFunctionBlockSerializedSize(factory);
registerFunctionToColumnTypeName(factory); registerFunctionToColumnTypeName(factory);
registerFunctionDumpColumnStructure(factory); registerFunctionDumpColumnStructure(factory);
registerFunctionDefaultValueOfArgumentType(factory); registerFunctionDefaultValueOfArgumentType(factory);

View File

@ -0,0 +1,16 @@
#include <IO/NullWriteBuffer.h>
namespace DB
{
NullWriteBuffer::NullWriteBuffer(size_t buf_size, char * existing_memory, size_t alignment)
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment)
{
}
void NullWriteBuffer::nextImpl()
{
}
}

View File

@ -0,0 +1,18 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
#include <boost/noncopyable.hpp>
namespace DB
{
/// Simply do nothing, can be used to measure amount of written bytes.
class NullWriteBuffer : public BufferWithOwnMemory<WriteBuffer>, boost::noncopyable
{
public:
NullWriteBuffer(size_t buf_size = 16<<10, char * existing_memory = nullptr, size_t alignment = false);
void nextImpl() override;
};
}

View File

@ -545,10 +545,12 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
// If this is a stub ATTACH query, read the query definition from the database // If this is a stub ATTACH query, read the query definition from the database
if (create.attach && !create.storage && !create.columns_list) if (create.attach && !create.storage && !create.columns_list)
{ {
bool if_not_exists = create.if_not_exists;
// Table SQL definition is available even if the table is detached // Table SQL definition is available even if the table is detached
auto query = context.getDatabase(create.database)->getCreateTableQuery(context, create.table); auto query = context.getDatabase(create.database)->getCreateTableQuery(context, create.table);
create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE
create.attach = true; create.attach = true;
create.if_not_exists = if_not_exists;
} }
String current_database = context.getCurrentDatabase(); String current_database = context.getCurrentDatabase();

View File

@ -57,6 +57,8 @@ void TabSeparatedRowOutputFormat::writeFieldDelimiter()
void TabSeparatedRowOutputFormat::writeRowEndDelimiter() void TabSeparatedRowOutputFormat::writeRowEndDelimiter()
{ {
if (format_settings.tsv.crlf_end_of_line)
writeChar('\r', out);
writeChar('\n', out); writeChar('\n', out);
} }

View File

@ -0,0 +1,6 @@
CREATE TABLE t (a Int) ENGINE = Log;
ATTACH TABLE t; -- { serverError 57 }
ATTACH TABLE IF NOT EXISTS t;
DETACH TABLE t;
ATTACH TABLE IF NOT EXISTS t;
EXISTS TABLE t;

View File

@ -0,0 +1,24 @@
UInt8 1
Nullable(UInt8) 2
UInt32 4
UInt64 8
Nullable(UInt64) 9
String 4
FixedString(32) 32
Enum8 1
Array 12
uniqCombinedState(100) 402
uniqCombinedState(10000) 81993
uniqCombinedState(100000) 81993
uniqCombinedState(1000000) 81993
uniqCombinedState(10000000) 81993
uniqCombined64State(10000000) 98505
String,UInt8 5
Block(UInt32) 16
Block(UInt32) 16

View File

@ -0,0 +1,29 @@
select 'UInt8', blockSerializedSize(0);
select 'Nullable(UInt8)', blockSerializedSize(toNullable(0));
select 'UInt32', blockSerializedSize(0xdeadbeaf);
select 'UInt64', blockSerializedSize(0xdeadbeafdead);
select 'Nullable(UInt64)', blockSerializedSize(toNullable(0xdeadbeafdead));
select '';
select 'String', blockSerializedSize('foo');
select 'FixedString(32)', blockSerializedSize(cast('foo', 'FixedString(32)'));
select '';
select 'Enum8', blockSerializedSize(cast('a' as Enum8('a' = 1, 'b' = 2)));
select '';
select 'Array', blockSerializedSize(['foo']);
select '';
select 'uniqCombinedState(100)', blockSerializedSize(uniqCombinedState(number)) from (select number from system.numbers limit 100);
select 'uniqCombinedState(10000)', blockSerializedSize(uniqCombinedState(number)) from (select number from system.numbers limit 10000);
select 'uniqCombinedState(100000)', blockSerializedSize(uniqCombinedState(number)) from (select number from system.numbers limit 100000);
select 'uniqCombinedState(1000000)', blockSerializedSize(uniqCombinedState(number)) from (select number from system.numbers limit 1000000);
select 'uniqCombinedState(10000000)', blockSerializedSize(uniqCombinedState(number)) from (select number from system.numbers limit 10000000);
select 'uniqCombined64State(10000000)', blockSerializedSize(uniqCombined64State(number)) from (select number from system.numbers limit 10000000);
select '';
select 'String,UInt8', blockSerializedSize('foo', 1);
select '';
select 'Block(UInt32)', blockSerializedSize(number) from numbers(2);

View File

@ -2,3 +2,7 @@
2,"world" 2,"world"
1,"hello" 1,"hello"
2,"world" 2,"world"
1 hello
2 world
1 hello
2 world

View File

@ -0,0 +1,8 @@
DROP TABLE IF EXISTS test_01073_crlf_end_of_line;
CREATE TABLE test_01073_crlf_end_of_line (value UInt8, word String) ENGINE = MergeTree() ORDER BY value;
INSERT INTO test_01073_crlf_end_of_line VALUES (1, 'hello'), (2, 'world');
SELECT * FROM test_01073_crlf_end_of_line FORMAT CSV SETTINGS output_format_csv_crlf_end_of_line = 1;
SELECT * FROM test_01073_crlf_end_of_line FORMAT CSV SETTINGS output_format_csv_crlf_end_of_line = 0;
SELECT * FROM test_01073_crlf_end_of_line FORMAT TSV SETTINGS output_format_tsv_crlf_end_of_line = 1;
SELECT * FROM test_01073_crlf_end_of_line FORMAT TSV SETTINGS output_format_tsv_crlf_end_of_line = 0;
DROP TABLE IF EXISTS test_01073_crlf_end_of_line;

View File

@ -1,8 +0,0 @@
DROP TABLE IF EXISTS test_01073_crlf_in_output_csv_format;
CREATE TABLE test_01073_crlf_in_output_csv_format (value UInt8, word String) ENGINE = MergeTree() ORDER BY value;
INSERT INTO test_01073_crlf_in_output_csv_format VALUES (1, 'hello'), (2, 'world');
SET output_format_csv_crlf_end_of_line = 1;
SELECT * FROM test_01073_crlf_in_output_csv_format FORMAT CSV;
SET output_format_csv_crlf_end_of_line = 0;
SELECT * FROM test_01073_crlf_in_output_csv_format FORMAT CSV;
DROP TABLE IF EXISTS test_01073_crlf_in_output_csv_format;

View File

@ -1,6 +1,7 @@
# docker build -t yandex/clickhouse-performance-comparison . # docker build -t yandex/clickhouse-performance-comparison .
FROM ubuntu:18.04 FROM ubuntu:18.04
ENV LANG=C.UTF-8
ENV TZ=Europe/Moscow ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone

View File

@ -130,7 +130,7 @@ function run_tests
rm left/performance/{IPv4,IPv6,modulo,parse_engine_file,number_formatting_formats,select_format}.xml ||: rm left/performance/{IPv4,IPv6,modulo,parse_engine_file,number_formatting_formats,select_format}.xml ||:
# Run the tests # Run the tests
for test in left/performance/*.xml for test in left/performance/${CHPC_TEST_GLOB:-*.xml}
do do
test_name=$(basename $test ".xml") test_name=$(basename $test ".xml")
echo test $test_name echo test $test_name

View File

@ -17,12 +17,16 @@ echo Reference tag is $ref_tag
ref_sha=$(cd ch && git rev-parse $ref_tag~0) ref_sha=$(cd ch && git rev-parse $ref_tag~0)
# Show what we're testing # Show what we're testing
echo Reference SHA is $ref_sha (
(cd ch && git log -1 --decorate $ref_sha) ||: echo Reference SHA is $ref_sha
echo (cd ch && git log -1 --decorate $ref_sha) ||:
echo SHA to test is $SHA_TO_TEST echo
(cd ch && git log -1 --decorate $SHA_TO_TEST) ||: ) | tee left-commit.txt
echo (
echo SHA to test is $SHA_TO_TEST
(cd ch && git log -1 --decorate $SHA_TO_TEST) ||:
echo
) | tee right-commit.txt
# Set python output encoding so that we can print queries with Russian letters. # Set python output encoding so that we can print queries with Russian letters.
export PYTHONIOENCODING=utf-8 export PYTHONIOENCODING=utf-8
@ -36,5 +40,5 @@ set -m
time ../compare.sh 0 $ref_sha $PR_TO_TEST $SHA_TO_TEST 2>&1 | ts | tee compare.log time ../compare.sh 0 $ref_sha $PR_TO_TEST $SHA_TO_TEST 2>&1 | ts | tee compare.log
set +m set +m
7z a /output/output.7z *.log *.tsv *.html 7z a /output/output.7z *.log *.tsv *.html *.txt
cp compare.log /output cp compare.log /output

View File

@ -77,18 +77,23 @@ def table_header(r):
def tsv_rows(n): def tsv_rows(n):
result = '' result = ''
with open(n) as fd: with open(n, encoding='utf-8') as fd:
for row in csv.reader(fd, delimiter="\t", quotechar='"'): for row in csv.reader(fd, delimiter="\t", quotechar='"'):
result += table_row(row) result += table_row(row)
return result return result
params = collections.defaultdict(str) params = collections.defaultdict(str)
params['header'] = "ClickHouse Performance Comparison" params['header'] = "ClickHouse Performance Comparison"
params['test_part'] = (table_template.format_map( params['test_part'] = (
collections.defaultdict(str, table_template.format(
caption = 'Tested commits',
header = table_header(['Old', 'New']),
rows = table_row([open('left-commit.txt').read(), open('right-commit.txt').read()])
) +
table_template.format(
caption = 'Changes in performance', caption = 'Changes in performance',
header = table_header(['Old, s', 'New, s', 'Relative difference (new&nbsp;-&nbsp;old)/old', 'Randomization distribution quantiles [5%,&nbsp;50%,&nbsp;95%]', 'Query']), header = table_header(['Old, s', 'New, s', 'Relative difference (new&nbsp;-&nbsp;old)/old', 'Randomization distribution quantiles [5%,&nbsp;50%,&nbsp;95%]', 'Query']),
rows = tsv_rows('changed-perf.tsv'))) + rows = tsv_rows('changed-perf.tsv')) +
table_template.format( table_template.format(
caption = 'Slow on client', caption = 'Slow on client',
header = table_header(['Client time, s', 'Server time, s', 'Ratio', 'Query']), header = table_header(['Client time, s', 'Server time, s', 'Ratio', 'Query']),

View File

@ -1,6 +1,6 @@
# Lazy # Lazy
Works like `Ordinary`, but keeps tables in RAM only `expiration_time_in_seconds` seconds after last access. Can be used only with \*Log tables. Keeps tables in RAM only `expiration_time_in_seconds` seconds after last access. Can be used only with \*Log tables.
It's optimized for storing many small \*Log tables, for which there is a long time interval between accesses. It's optimized for storing many small \*Log tables, for which there is a long time interval between accesses.

View File

@ -2,7 +2,7 @@
[CatBoost](https://catboost.ai) is a free and open-source gradient boosting library developed at [Yandex](https://yandex.com/company/) for machine learning. [CatBoost](https://catboost.ai) is a free and open-source gradient boosting library developed at [Yandex](https://yandex.com/company/) for machine learning.
With this instruction, you will learn to apply pre-trained models in ClickHouse: as a result, you run the model inference from SQL. With this instruction, you will learn to apply pre-trained models in ClickHouse by running model inference from SQL.
To apply a CatBoost model in ClickHouse: To apply a CatBoost model in ClickHouse:

View File

@ -797,6 +797,10 @@ For CSV input format enables or disables parsing of unquoted `NULL` as literal (
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}
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 quorum writes.
@ -1067,7 +1071,7 @@ 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](../performance/sampling_query_profiler.md). Real clock timer counts wall-clock time. 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.
Possible values: Possible values:
@ -1090,7 +1094,7 @@ See also:
## 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](../performance/sampling_query_profiler.md). This timer counts only CPU time. 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: Possible values:

View File

@ -312,7 +312,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. 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](../select.md#select-where)). Any condition can be specified as an argument (as in [WHERE](../../query_language/select.md#select-where)).
The conditions, except the first, apply in pairs: the result of the second will be true if the first and second are true, of the third if the first and fird are true, etc. The conditions, except the first, apply in pairs: the result of the second will be true if the first and second are true, of the third if the first and fird are true, etc.

View File

@ -371,7 +371,7 @@ dateDiff('unit', startdate, enddate, [timezone])
- `startdate` — The first time value to compare. [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md). - `startdate` — The first time value to compare. [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md).
- `enddate` — The second time value to compare. [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md). - `enddate` — The second time value to compare. [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md).
- `timezone` — Optional parameter. If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified. [Time Zone](../../data_types/datetime.md#time-zones). - `timezone` — Optional parameter. If specified, it is applied to both `startdate` and `enddate`. If not specified, timezones of `startdate` and `enddate` are used. If they are not the same, the result is unspecified.
**Returned value** **Returned value**

View File

@ -20,7 +20,7 @@ dictGetOrDefault('dict_name', 'attr_name', id_expr, default_value_expr)
**Returned value** **Returned value**
- If ClickHouse parses the attribute successfully in the [attribute's data type](../dicts/external_dicts_dict_structure.md#ext_dict_structure-attributes), functions return the value of the dictionary attribute that corresponds to `id_expr`. - If ClickHouse parses the attribute successfully in the [attribute's data type](../../query_language/dicts/external_dicts_dict_structure.md#ext_dict_structure-attributes), functions return the value of the dictionary attribute that corresponds to `id_expr`.
- If there is no the key, corresponding to `id_expr`, in the dictionary, then: - If there is no the key, corresponding to `id_expr`, in the dictionary, then:
- `dictGet` returns the content of the `<null_value>` element specified for the attribute in the dictionary configuration. - `dictGet` returns the content of the `<null_value>` element specified for the attribute in the dictionary configuration.
@ -183,7 +183,7 @@ dictGet[Type]OrDefault('dict_name', 'attr_name', id_expr, default_value_expr)
**Returned value** **Returned value**
- If ClickHouse parses the attribute successfully in the [attribute's data type](../dicts/external_dicts_dict_structure.md#ext_dict_structure-attributes), functions return the value of the dictionary attribute that corresponds to `id_expr`. - If ClickHouse parses the attribute successfully in the [attribute's data type](../../query_language/dicts/external_dicts_dict_structure.md#ext_dict_structure-attributes), functions return the value of the dictionary attribute that corresponds to `id_expr`.
- If there is no requested `id_expr` in the dictionary then: - If there is no requested `id_expr` in the dictionary then:
- `dictGet[Type]` returns the content of the `<null_value>` element specified for the attribute in the dictionary configuration. - `dictGet[Type]` returns the content of the `<null_value>` element specified for the attribute in the dictionary configuration.

View File

@ -603,6 +603,34 @@ SELECT getSizeOfEnumType( CAST('a' AS Enum8('a' = 1, 'b' = 2) ) ) AS x
└───┘ └───┘
``` ```
## blockSerializedSize
Returns size on disk (without taking into account compression).
```sql
blockSerializedSize(value[, value[, ...]])
```
**Parameters:**
- `value` — Any value.
**Returned values**
- The number of bytes that will be written to disk for block of values (without compression).
**Example**
```sql
SELECT blockSerializedSize(maxState(1)) as x
```
```text
┌─x─┐
│ 2 │
└───┘
```
## toColumnTypeName ## toColumnTypeName
Returns the name of the class that represents the data type of the column in RAM. Returns the name of the class that represents the data type of the column in RAM.
@ -955,7 +983,7 @@ So, result of function depends on partition of data to blocks and on order of da
## joinGet {#joinget} ## joinGet {#joinget}
The function lets you extract data from the table the same way as from a [dictionary](../dicts/index.md). The function lets you extract data from the table the same way as from a [dictionary](../../query_language/dicts/index.md).
Gets data from [Join](../../operations/table_engines/join.md#creating-a-table) tables using the specified join key. Gets data from [Join](../../operations/table_engines/join.md#creating-a-table) tables using the specified join key.

View File

@ -17,7 +17,7 @@ For a case-insensitive search, use the function `positionCaseInsensitiveUTF8`.
## multiSearchAllPositions {#multiSearchAllPositions} ## multiSearchAllPositions {#multiSearchAllPositions}
The same as [position](#position) but returns `Array` of positions (in bytes) of the found corresponding substrings in the string. Positions are indexed starting from 1. The same as [position](string_search_functions.md#position) but returns `Array` of positions (in bytes) of the found corresponding substrings in the string. Positions are indexed starting from 1.
The search is performed on sequences of bytes without respect to string encoding and collation. The search is performed on sequences of bytes without respect to string encoding and collation.

View File

@ -704,7 +704,7 @@ For `ON`, `WHERE`, and `GROUP BY` clauses:
- Arbitrary expressions cannot be used in `ON`, `WHERE`, and `GROUP BY` clauses, but you can define an expression in a `SELECT` clause and then use it in these clauses via an alias. - Arbitrary expressions cannot be used in `ON`, `WHERE`, and `GROUP BY` clauses, but you can define an expression in a `SELECT` clause and then use it in these clauses via an alias.
### WHERE Clause ### WHERE Clause {#select-where}
If there is a WHERE clause, it must contain an expression with the UInt8 type. This is usually an expression with comparison and logical operators. If there is a WHERE clause, it must contain an expression with the UInt8 type. This is usually an expression with comparison and logical operators.
This expression will be used for filtering data before all other transformations. This expression will be used for filtering data before all other transformations.

View File

@ -1,6 +1,6 @@
<div dir="rtl" markdown="1"> <div dir="rtl" markdown="1">
# DateTime # DateTime {#data_type-datetime}
تاریخ با ساعت 4 بایت به صورت Unix timestamp ذخیره می کند (unsigned). به شما اجازه ی ذخیره سازی در محدوده ی تایپ Date را می دهد. حداقل مقدار در خروجی 0000-00-00 00:00:00 می باشد. زمان با دقت تا یک ثانیه ذخیره می شود. تاریخ با ساعت 4 بایت به صورت Unix timestamp ذخیره می کند (unsigned). به شما اجازه ی ذخیره سازی در محدوده ی تایپ Date را می دهد. حداقل مقدار در خروجی 0000-00-00 00:00:00 می باشد. زمان با دقت تا یک ثانیه ذخیره می شود.

View File

@ -20,6 +20,7 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su
##ﺩﻮﺟﻮﻣ ﺐﺼﻧ ﯼﺎﻫ ﻪﻨﯾﺰﮔ ##ﺩﻮﺟﻮﻣ ﺐﺼﻧ ﯼﺎﻫ ﻪﻨﯾﺰﮔ
<a name="install-from-deb-packages"></a>
### نصب از طریق پکیج های Debian/Ubuntu {#from-deb-packages} ### نصب از طریق پکیج های Debian/Ubuntu {#from-deb-packages}
در فایل `/etc/apt/sources.list` (یا در یک فایل جدا `/etc/apt/sources.list.d/clickhouse.list`)، Repo زیر را اضافه کنید: در فایل `/etc/apt/sources.list` (یا در یک فایل جدا `/etc/apt/sources.list.d/clickhouse.list`)، Repo زیر را اضافه کنید:

View File

@ -0,0 +1 @@
../../../en/operations/performance/sampling_query_profiler.md

View File

@ -0,0 +1 @@
../../../en/operations/performance/sampling_query_profiler.md

View File

@ -0,0 +1 @@
../../../en/operations/performance/sampling_query_profiler.md

View File

@ -604,6 +604,27 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat
</query_thread_log> </query_thread_log>
``` ```
## trace_log {#server_settings-trace_log}
Settings for the [trace_log](../system_tables.md#system_tables-trace_log) system table operation.
Parameters:
- `database` — Database for storing a table.
- `table` — Table name.
- `partition_by` — [Custom partitioning key](../../operations/table_engines/custom_partitioning_key.md) for a system table.
- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table.
The default server configuration file `config.xml` contains the following settings section:
```xml
<trace_log>
<database>system</database>
<table>trace_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</trace_log>
```
## remote_servers {#server_settings_remote_servers} ## remote_servers {#server_settings_remote_servers}

View File

@ -493,7 +493,7 @@ ClickHouse использует этот параметр при чтении д
Установка логирования запроса. Установка логирования запроса.
Запросы, переданные в ClickHouse с этой установкой, логируются согласно правилам конфигурационного параметра сервера [query_log](../server_settings/settings.md#server_settings-query-log). Запросы, переданные в ClickHouse с этой установкой, логируются согласно правилам конфигурационного параметра сервера [query_log](../../operations/server_settings/settings.md#server_settings-query-log).
Пример: Пример:
@ -780,6 +780,10 @@ load_balancing = first_or_random
Использовать в качестве разделителя строк для CSV формата CRLF (DOS/Windows стиль) вместо LF (Unix стиль). Использовать в качестве разделителя строк для CSV формата CRLF (DOS/Windows стиль) вместо LF (Unix стиль).
## output_format_tsv_crlf_end_of_line {#settings-output_format_tsv_crlf_end_of_line}
Использовать в качестве разделителя строк для TSV формата CRLF (DOC/Windows стиль) вместо LF (Unix стиль).
## insert_quorum {#settings-insert_quorum} ## insert_quorum {#settings-insert_quorum}
Включает кворумную запись. Включает кворумную запись.
@ -1008,5 +1012,66 @@ ClickHouse генерирует исключение
Значение по умолчанию: 0. Значение по умолчанию: 0.
## 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.
Possible values:
- Positive integer number, in nanoseconds.
Recommended values:
- 10000000 (100 times a second) nanoseconds and less for single queries.
- 1000000000 (once a second) for cluster-wide profiling.
- 0 for turning off the timer.
Type: [UInt64](../../data_types/int_uint.md).
Default value: 1000000000 nanoseconds (once a second).
See also:
- System table [trace_log](../../operations/system_tables.md#system_tables-trace_log)
## 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.
Recommended values:
- 10000000 (100 times a second) nanosecods and more for for single queries.
- 1000000000 (once a second) for cluster-wide profiling.
- 0 for turning off the timer.
Type: [UInt64](../../data_types/int_uint.md).
Default value: 1000000000 nanoseconds.
See also:
- System table [trace_log](../../operations/system_tables.md#system_tables-trace_log)
## allow_introspection_functions {#settings-allow_introspection_functions}
Enables of disables [introspections functions](../../query_language/functions/introspection.md) for query profiling.
Possible values:
- 1 — Introspection functions enabled.
- 0 — Introspection functions disabled.
Default value: 0.
**See Also**
- [Sampling Query Profiler](../performance/sampling_query_profiler.md)
- System table [trace_log](../../operations/system_tables.md#system_tables-trace_log)
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) <!--hide--> [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) <!--hide-->

View File

@ -474,6 +474,91 @@ ClickHouse создаёт таблицу только в том случае, к
Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query_log](server_settings/settings.md#server_settings-query-log) (параметр `partition_by`). Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query_log](server_settings/settings.md#server_settings-query-log) (параметр `partition_by`).
## 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.
!!! note "Note"
The table doesn't contain input data for `INSERT` queries.
ClickHouse creates this table only if the [query_log](server_settings/settings.md#server_settings-query-log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in.
To enable query logging, set the [log_queries](settings/settings.md#settings-log-queries) parameter to 1. For details, see the [Settings](settings/settings.md) section.
The `system.query_log` table registers two kinds of queries:
1. Initial queries that were run directly by the client.
2. Child queries that were initiated by other queries (for distributed query execution). For these types of queries, information about the parent queries is shown in the `initial_*` columns.
Columns:
- `type` (`Enum8`) — Type of event that occurred when executing the query. Values:
- `'QueryStart' = 1` — Successful start of query execution.
- `'QueryFinish' = 2` — Successful end of query execution.
- `'ExceptionBeforeStart' = 3` — Exception before the start of query execution.
- `'ExceptionWhileProcessing' = 4` — Exception during the query execution.
- `event_date` (Date) — Query starting date.
- `event_time` (DateTime) — Query starting time.
- `query_start_time` (DateTime) — Start time of query execution.
- `query_duration_ms` (UInt64) — Duration of query execution.
- `read_rows` (UInt64) — Number of read rows.
- `read_bytes` (UInt64) — Number of read bytes.
- `written_rows` (UInt64) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0.
- `written_bytes` (UInt64) — For `INSERT` queries, the number of written bytes. For other queries, the column value is 0.
- `result_rows` (UInt64) — Number of rows in the result.
- `result_bytes` (UInt64) — Number of bytes in the result.
- `memory_usage` (UInt64) — Memory consumption by the query.
- `query` (String) — Query string.
- `exception` (String) — Exception message.
- `stack_trace` (String) — Stack trace (a list of methods called before the error occurred). An empty string, if the query is completed successfully.
- `is_initial_query` (UInt8) — Query type. Possible values:
- 1 — Query was initiated by the client.
- 0 — Query was initiated by another query for distributed query execution.
- `user` (String) — Name of the user who initiated the current query.
- `query_id` (String) — ID of the query.
- `address` (IPv6) — IP address that was used to make the query.
- `port` (UInt16) — The client port that was used to make the query.
- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution).
- `initial_query_id` (String) — ID of the initial query (for distributed query execution).
- `initial_address` (IPv6) — IP address that the parent query was launched from.
- `initial_port` (UInt16) — The client port that was used to make the parent query.
- `interface` (UInt8) — Interface that the query was initiated from. Possible values:
- 1 — TCP.
- 2 — HTTP.
- `os_user` (String) — OS's username who runs [clickhouse-client](../interfaces/cli.md).
- `client_hostname` (String) — Hostname of the client machine where the [clickhouse-client](../interfaces/cli.md) or another TCP client is run.
- `client_name` (String) — The [clickhouse-client](../interfaces/cli.md) or another TCP client name.
- `client_revision` (UInt32) — Revision of the [clickhouse-client](../interfaces/cli.md) or another TCP client.
- `client_version_major` (UInt32) — Major version of the [clickhouse-client](../interfaces/cli.md) or another TCP client.
- `client_version_minor` (UInt32) — Minor version of the [clickhouse-client](../interfaces/cli.md) or another TCP client.
- `client_version_patch` (UInt32) — Patch component of the [clickhouse-client](../interfaces/cli.md) or another TCP client version.
- `http_method` (UInt8) — HTTP method that initiated the query. Possible values:
- 0 — The query was launched from the TCP interface.
- 1 — `GET` method was used.
- 2 — `POST` method was used.
- `http_user_agent` (String) — The `UserAgent` header passed in the HTTP request.
- `quota_key` (String) — The "quota key" specified in the [quotas](quotas.md) setting (see `keyed`).
- `revision` (UInt32) — ClickHouse revision.
- `thread_numbers` (Array(UInt32)) — Number of threads that are participating in query execution.
- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics. The description of them could be found in the table [system.events](#system_tables-events)
- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics that are listed in the `ProfileEvents.Names` column.
- `Settings.Names` (Array(String)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1.
- `Settings.Values` (Array(String)) — Values of settings that are listed in the `Settings.Names` column.
Each query creates one or two rows in the `query_log` table, depending on the status of the query:
1. If the query execution is successful, two events with types 1 and 2 are created (see the `type` column).
2. If an error occurred during query processing, two events with types 1 and 4 are created.
3. If an error occurred before launching the query, a single event with type 3 is created.
By default, logs are added to the table at intervals of 7.5 seconds. You can set this interval in the [query_log](server_settings/settings.md#server_settings-query-log) server setting (see the `flush_interval_milliseconds` parameter). To flush the logs forcibly from the memory buffer into the table, use the `SYSTEM FLUSH LOGS` query.
When the table is deleted manually, it will be automatically created on the fly. Note that all the previous logs will be deleted.
!!! note
The storage period for logs is unlimited. Logs aren't automatically deleted from the table. You need to organize the removal of outdated logs yourself.
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}
Содержит информацию о каждом потоке выполняемых запросов. Содержит информацию о каждом потоке выполняемых запросов.
@ -538,6 +623,48 @@ ClickHouse создаёт таблицу только в том случае, к
Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) (параметр `partition_by`). Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) (параметр `partition_by`).
## system.trace_log {#system_tables-trace_log}
Contains stack traces collected by the sampling query profiler.
ClickHouse creates this table when the [trace_log](server_settings/settings.md#server_settings-trace_log) server configuration section is set. Also the [query_profiler_real_time_period_ns](settings/settings.md#query_profiler_real_time_period_ns) and [query_profiler_cpu_time_period_ns](settings/settings.md#query_profiler_cpu_time_period_ns) settings should be set.
To analyze logs, use the `addressToLine`, `addressToSymbol` and `demangle` introspection functions.
Columns:
- `event_date`([Date](../data_types/date.md)) — Date of sampling moment.
- `event_time`([DateTime](../data_types/datetime.md)) — Timestamp of sampling moment.
- `revision`([UInt32](../data_types/int_uint.md)) — ClickHouse server build revision.
When connecting to server by `clickhouse-client`, you see the string similar to `Connected to ClickHouse server version 19.18.1 revision 54429.`. This field contains the `revision`, but not the `version` of a server.
- `timer_type`([Enum8](../data_types/enum.md)) — Timer type:
- `Real` represents wall-clock time.
- `CPU` represents CPU time.
- `thread_number`([UInt32](../data_types/int_uint.md)) — Thread identifier.
- `query_id`([String](../data_types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) system table.
- `trace`([Array(UInt64)](../data_types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process.
**Example**
```sql
SELECT * FROM system.trace_log LIMIT 1 \G
```
```text
Row 1:
──────
event_date: 2019-11-15
event_time: 2019-11-15 15:09:38
revision: 54428
timer_type: Real
thread_number: 48
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}
Содержит информацию и статус для реплицируемых таблиц, расположенных на локальном сервере. Содержит информацию и статус для реплицируемых таблиц, расположенных на локальном сервере.

View File

@ -189,7 +189,7 @@ dateDiff('unit', startdate, enddate, [timezone])
- `startdate` — Первая дата. [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md). - `startdate` — Первая дата. [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md).
- `enddate` — Вторая дата. [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md). - `enddate` — Вторая дата. [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md).
- `timezone` — Опциональный параметр. Если определен, применяется к обоим значениям: `startdate` и `enddate`. Если не определен, используются часовые пояса `startdate` и `enddate`. Если часовые пояса не совпадают, вернется неожидаемый результат. [Часовые пояса](../../data_types/datetime.md#time-zones). - `timezone` — Опциональный параметр. Если определен, применяется к обоим значениям: `startdate` и `enddate`. Если не определен, используются часовые пояса `startdate` и `enddate`. Если часовые пояса не совпадают, вернется неожидаемый результат.
**Возвращаемое значение** **Возвращаемое значение**

View File

@ -0,0 +1 @@
../../../en/query_language/functions/introspection.md

View File

@ -934,7 +934,7 @@ SELECT formatReadableSize(filesystemCapacity()) AS "Capacity", toTypeName(filesy
## joinGet {#joinget} ## joinGet {#joinget}
Функция позволяет извлекать данные из таблицы таким же образом как из [словаря](../dicts/index.md). Функция позволяет извлекать данные из таблицы таким же образом как из [словаря](../../query_language/dicts/index.md).
Получает данные из таблиц [Join](../../operations/table_engines/join.md#creating-a-table) по ключу. Получает данные из таблиц [Join](../../operations/table_engines/join.md#creating-a-table) по ключу.

View File

@ -2,7 +2,7 @@
Во всех функциях, поиск регистрозависимый по умолчанию. Существуют варианты функций для регистронезависимого поиска. Во всех функциях, поиск регистрозависимый по умолчанию. Существуют варианты функций для регистронезависимого поиска.
## position(haystack, needle) ## position(haystack, needle) {#position}
Поиск подстроки `needle` в строке `haystack`. Поиск подстроки `needle` в строке `haystack`.
Возвращает позицию (в байтах) найденной подстроки, начиная с 1, или 0, если подстрока не найдена. Возвращает позицию (в байтах) найденной подстроки, начиная с 1, или 0, если подстрока не найдена.

View File

@ -68,6 +68,6 @@ SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123')
## Смотрите также ## Смотрите также
- [Движок таблиц 'MySQL'](../../operations/table_engines/mysql.md) - [Движок таблиц 'MySQL'](../../operations/table_engines/mysql.md)
- [Использование MySQL как источника данных для внешнего словаря](../dicts/external_dicts_dict_sources.md#dicts-external_dicts_dict_sources-mysql) - [Использование MySQL как источника данных для внешнего словаря](../../query_language/dicts/external_dicts_dict_sources.md#dicts-external_dicts_dict_sources-mysql)
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/table_functions/mysql/) <!--hide--> [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/table_functions/mysql/) <!--hide-->

View File

@ -199,7 +199,7 @@ nav:
- 'Quotas': 'operations/quotas.md' - 'Quotas': 'operations/quotas.md'
- 'System Tables': 'operations/system_tables.md' - 'System Tables': 'operations/system_tables.md'
- 'Optimizing Performance': - 'Optimizing Performance':
- 'Query Profiling': operations/performance/sampling_query_profiler.md - 'Query Profiling': 'operations/performance/sampling_query_profiler.md'
- 'Testing Hardware': 'operations/performance_test.md' - 'Testing Hardware': 'operations/performance_test.md'
- 'Server Configuration Parameters': - 'Server Configuration Parameters':
- 'Introduction': 'operations/server_settings/index.md' - 'Introduction': 'operations/server_settings/index.md'

View File

@ -193,6 +193,8 @@ nav:
- 'Configuration Files': 'operations/configuration_files.md' - 'Configuration Files': 'operations/configuration_files.md'
- 'Quotas': 'operations/quotas.md' - 'Quotas': 'operations/quotas.md'
- 'System Tables': 'operations/system_tables.md' - 'System Tables': 'operations/system_tables.md'
- 'Optimizing Performance':
- 'Query Profiling': 'operations/performance/sampling_query_profiler.md'
- 'Testing Hardware': 'operations/performance_test.md' - 'Testing Hardware': 'operations/performance_test.md'
- 'Server Configuration Parameters': - 'Server Configuration Parameters':
- 'Introduction': 'operations/server_settings/index.md' - 'Introduction': 'operations/server_settings/index.md'

View File

@ -197,6 +197,8 @@ nav:
- 'Configuration Files': 'operations/configuration_files.md' - 'Configuration Files': 'operations/configuration_files.md'
- 'Quotas': 'operations/quotas.md' - 'Quotas': 'operations/quotas.md'
- 'System Tables': 'operations/system_tables.md' - 'System Tables': 'operations/system_tables.md'
- 'Optimizing Performance':
- 'Query Profiling': 'operations/performance/sampling_query_profiler.md'
- 'Testing Hardware': 'operations/performance_test.md' - 'Testing Hardware': 'operations/performance_test.md'
- 'Server Configuration Parameters': - 'Server Configuration Parameters':
- 'Introduction': 'operations/server_settings/index.md' - 'Introduction': 'operations/server_settings/index.md'

View File

@ -122,6 +122,7 @@ nav:
- 'Функции для работы с географическими координатами': 'query_language/functions/geo.md' - 'Функции для работы с географическими координатами': 'query_language/functions/geo.md'
- 'Функции c Nullable аргументами': 'query_language/functions/functions_for_nulls.md' - 'Функции c Nullable аргументами': 'query_language/functions/functions_for_nulls.md'
- 'Функции машинного обучения': 'query_language/functions/machine_learning_functions.md' - 'Функции машинного обучения': 'query_language/functions/machine_learning_functions.md'
- 'Функции для интроспекции': 'query_language/functions/introspection.md'
- 'Прочие функции': 'query_language/functions/other_functions.md' - 'Прочие функции': 'query_language/functions/other_functions.md'
- 'Агрегатные функции': - 'Агрегатные функции':
- 'Введение': 'query_language/agg_functions/index.md' - 'Введение': 'query_language/agg_functions/index.md'
@ -197,6 +198,8 @@ nav:
- 'Конфигурационные файлы': 'operations/configuration_files.md' - 'Конфигурационные файлы': 'operations/configuration_files.md'
- 'Квоты': 'operations/quotas.md' - 'Квоты': 'operations/quotas.md'
- 'Системные таблицы': 'operations/system_tables.md' - 'Системные таблицы': 'operations/system_tables.md'
- 'Оптимизация производительности':
- 'Профилирование запросов': 'operations/performance/sampling_query_profiler.md'
- 'Тестирование оборудования': 'operations/performance_test.md' - 'Тестирование оборудования': 'operations/performance_test.md'
- 'Конфигурационные параметры сервера': - 'Конфигурационные параметры сервера':
- 'Введение': 'operations/server_settings/index.md' - 'Введение': 'operations/server_settings/index.md'

View File

@ -192,7 +192,9 @@ nav:
- '配置文件': 'operations/configuration_files.md' - '配置文件': 'operations/configuration_files.md'
- '配额': 'operations/quotas.md' - '配额': 'operations/quotas.md'
- '系统表': 'operations/system_tables.md' - '系统表': 'operations/system_tables.md'
- 'Testing Hardware': 'operations/performance_test.md' - '优化性能':
- '查询分析': 'operations/performance/sampling_query_profiler.md'
- '测试硬件': 'operations/performance_test.md'
- 'Server参数配置': - 'Server参数配置':
- '介绍': 'operations/server_settings/index.md' - '介绍': 'operations/server_settings/index.md'
- 'Server参数说明': 'operations/server_settings/settings.md' - 'Server参数说明': 'operations/server_settings/settings.md'

207
docs/tools/output.md Normal file
View File

@ -0,0 +1,207 @@
What is ClickHouse?
===================
ClickHouse is a column-oriented database management system (DBMS) for
online analytical processing of queries (OLAP).
In a "normal" row-oriented DBMS, data is stored in this order:
Row WatchID JavaEnable Title GoodEvent EventTime
----- ------------- ------------ -------------------- ----------- ---------------------
\#0 89354350662 1 Investor Relations 1 2016-05-18 05:19:20
\#1 90329509958 0 Contact us 1 2016-05-18 08:10:20
\#2 89953706054 1 Mission 1 2016-05-18 07:38:00
\#N ... ... ... ... ...
In other words, all the values related to a row are physically stored
next to each other.
Examples of a row-oriented DBMS are MySQL, Postgres, and MS SQL Server.
{: .grey }
In a column-oriented DBMS, data is stored like this:
Row: \#0 \#1 \#2 \#N
------------- --------------------- --------------------- --------------------- -----
WatchID: 89354350662 90329509958 89953706054 ...
JavaEnable: 1 0 1 ...
Title: Investor Relations Contact us Mission ...
GoodEvent: 1 1 1 ...
EventTime: 2016-05-18 05:19:20 2016-05-18 08:10:20 2016-05-18 07:38:00 ...
These examples only show the order that data is arranged in. The values
from different columns are stored separately, and data from the same
column is stored together.
Examples of a column-oriented DBMS: Vertica, Paraccel (Actian Matrix and
Amazon Redshift), Sybase IQ, Exasol, Infobright, InfiniDB, MonetDB
(VectorWise and Actian Vector), LucidDB, SAP HANA, Google Dremel, Google
PowerDrill, Druid, and kdb+. {: .grey }
Different orders for storing data are better suited to different
scenarios. The data access scenario refers to what queries are made, how
often, and in what proportion; how much data is read for each type of
query -- rows, columns, and bytes; the relationship between reading and
updating data; the working size of the data and how locally it is used;
whether transactions are used, and how isolated they are; requirements
for data replication and logical integrity; requirements for latency and
throughput for each type of query, and so on.
The higher the load on the system, the more important it is to customize
the system set up to match the requirements of the usage scenario, and
the more fine grained this customization becomes. There is no system
that is equally well-suited to significantly different scenarios. If a
system is adaptable to a wide set of scenarios, under a high load, the
system will handle all the scenarios equally poorly, or will work well
for just one or few of possible scenarios.
Key Properties of the OLAP scenario
-----------------------------------
- The vast majority of requests are for read access.
- Data is updated in fairly large batches (\> 1000 rows), not by
single rows; or it is not updated at all.
- Data is added to the DB but is not modified.
- For reads, quite a large number of rows are extracted from the DB,
but only a small subset of columns.
- Tables are "wide," meaning they contain a large number of columns.
- Queries are relatively rare (usually hundreds of queries per server
or less per second).
- For simple queries, latencies around 50 ms are allowed.
- Column values are fairly small: numbers and short strings (for
example, 60 bytes per URL).
- Requires high throughput when processing a single query (up to
billions of rows per second per server).
- Transactions are not necessary.
- Low requirements for data consistency.
- There is one large table per query. All tables are small, except for
one.
- A query result is significantly smaller than the source data. In
other words, data is filtered or aggregated, so the result fits in a
single server's RAM.
It is easy to see that the OLAP scenario is very different from other
popular scenarios (such as OLTP or Key-Value access). So it doesn't make
sense to try to use OLTP or a Key-Value DB for processing analytical
queries if you want to get decent performance. For example, if you try
to use MongoDB or Redis for analytics, you will get very poor
performance compared to OLAP databases.
Why Column-Oriented Databases Work Better in the OLAP Scenario
--------------------------------------------------------------
Column-oriented databases are better suited to OLAP scenarios: they are
at least 100 times faster in processing most queries. The reasons are
explained in detail below, but the fact is easier to demonstrate
visually:
**Row-oriented DBMS**
![Row-oriented](images/row_oriented.gif#)
**Column-oriented DBMS**
![Column-oriented](images/column_oriented.gif#)
See the difference?
### Input/output
1. For an analytical query, only a small number of table columns need
to be read. In a column-oriented database, you can read just the
data you need. For example, if you need 5 columns out of 100, you
can expect a 20-fold reduction in I/O.
2. Since data is read in packets, it is easier to compress. Data in
columns is also easier to compress. This further reduces the I/O
volume.
3. Due to the reduced I/O, more data fits in the system cache.
For example, the query "count the number of records for each advertising
platform" requires reading one "advertising platform ID" column, which
takes up 1 byte uncompressed. If most of the traffic was not from
advertising platforms, you can expect at least 10-fold compression of
this column. When using a quick compression algorithm, data
decompression is possible at a speed of at least several gigabytes of
uncompressed data per second. In other words, this query can be
processed at a speed of approximately several billion rows per second on
a single server. This speed is actually achieved in practice.
<details>
<summary>Example</summary>
$ clickhouse-client
ClickHouse client version 0.0.52053.
Connecting to localhost:9000.
Connected to ClickHouse server version 0.0.52053.
:) SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20
SELECT
CounterID,
count()
FROM hits
GROUP BY CounterID
ORDER BY count() DESC
LIMIT 20
┌─CounterID─┬──count()─┐
│ 114208 │ 56057344 │
│ 115080 │ 51619590 │
│ 3228 │ 44658301 │
│ 38230 │ 42045932 │
│ 145263 │ 42042158 │
│ 91244 │ 38297270 │
│ 154139 │ 26647572 │
│ 150748 │ 24112755 │
│ 242232 │ 21302571 │
│ 338158 │ 13507087 │
│ 62180 │ 12229491 │
│ 82264 │ 12187441 │
│ 232261 │ 12148031 │
│ 146272 │ 11438516 │
│ 168777 │ 11403636 │
│ 4120072 │ 11227824 │
│ 10938808 │ 10519739 │
│ 74088 │ 9047015 │
│ 115079 │ 8837972 │
│ 337234 │ 8205961 │
└───────────┴──────────┘
20 rows in set. Elapsed: 0.153 sec. Processed 1.00 billion rows, 4.00 GB (6.53 billion rows/s., 26.10 GB/s.)
:)
</details>
### CPU
Since executing a query requires processing a large number of rows, it
helps to dispatch all operations for entire vectors instead of for
separate rows, or to implement the query engine so that there is almost
no dispatching cost. If you don't do this, with any half-decent disk
subsystem, the query interpreter inevitably stalls the CPU. It makes
sense to both store data in columns and process it, when possible, by
columns.
There are two ways to do this:
1. A vector engine. All operations are written for vectors, instead of
for separate values. This means you don't need to call operations
very often, and dispatching costs are negligible. Operation code
contains an optimized internal cycle.
2. Code generation. The code generated for the query has all the
indirect calls in it.
This is not done in "normal" databases, because it doesn't make sense
when running simple queries. However, there are exceptions. For example,
MemSQL uses code generation to reduce latency when processing SQL
queries. (For comparison, analytical DBMSs require optimization of
throughput, not latency.)
Note that for CPU efficiency, the query language must be declarative
(SQL or MDX), or at least a vector (J, K). The query should only contain
implicit loops, allowing for optimization.
[Original article](https://clickhouse.tech/docs/en/) <!--hide-->

View File

@ -27,12 +27,13 @@ then
git config user.email "robot-clickhouse@yandex-team.ru" git config user.email "robot-clickhouse@yandex-team.ru"
git config user.name "robot-clickhouse" git config user.name "robot-clickhouse"
git rm -rf * git rm -rf *
git commit -a -m "wipe old release"
cp -R "${BUILD_DIR}"/* . cp -R "${BUILD_DIR}"/* .
echo -n "test.clickhouse.tech" > CNAME echo -n "test.clickhouse.tech" > CNAME
echo -n "" > README.md echo -n "" > README.md
echo -n "" > ".nojekyll"
cp "${BASE_DIR}/../../LICENSE" . cp "${BASE_DIR}/../../LICENSE" .
git add * git add *
git add ".nojekyll"
git commit -a -m "add new release at $(date)" git commit -a -m "add new release at $(date)"
git push origin master git push origin master
cd "${BUILD_DIR}" cd "${BUILD_DIR}"

21
docs/tools/translate.py Executable file
View File

@ -0,0 +1,21 @@
#!/usr/bin/env python
from __future__ import print_function
import sys
import pprint
import googletrans
import pandocfilters
translator = googletrans.Translator()
def translate(key, value, format, _):
if key == 'Str':
print(value.encode('utf8'), file=sys.stderr)
return
[meta, contents] = value
cls = getattr(pandocfilters, key)
return cls(meta, translator.translate(contents, dest='es'))
if __name__ == "__main__":
pandocfilters.toJSONFilter(translate)

View File

@ -2,6 +2,7 @@ import contextlib
import multiprocessing import multiprocessing
import os import os
import shutil import shutil
import sys
import tempfile import tempfile
import threading import threading
@ -26,9 +27,15 @@ def autoremoved_file(path):
def run_function_in_parallel(func, args_list, threads=False): def run_function_in_parallel(func, args_list, threads=False):
processes = [] processes = []
exit_code = 0
for task in args_list: for task in args_list:
cls = threading.Thread if threads else multiprocessing.Process cls = threading.Thread if threads else multiprocessing.Process
processes.append(cls(target=func, args=task)) processes.append(cls(target=func, args=task))
processes[-1].start() processes[-1].start()
for process in processes: for process in processes:
process.join() process.join()
if not threads:
if process.exitcode and not exit_code:
exit_code = process.exitcode
if exit_code:
sys.exit(exit_code)

View File

@ -12,6 +12,7 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not
##可用的安装选项 ##可用的安装选项
<a name="install-from-deb-packages"></a>
### 为Debian/Ubuntu安装 {#from-deb-packages} ### 为Debian/Ubuntu安装 {#from-deb-packages}
在`/etc/apt/sources.list` (或创建`/etc/apt/sources.list.d/clickhouse.list`文件)中添加仓库: 在`/etc/apt/sources.list` (或创建`/etc/apt/sources.list.d/clickhouse.list`文件)中添加仓库:

View File

@ -0,0 +1 @@
../../../en/operations/performance/sampling_query_profiler.md

View File

@ -104,7 +104,7 @@ Defines storage time for values. Can be specified only for MergeTree-family tabl
### Column Compression Codecs {#codecs} ### Column Compression Codecs {#codecs}
By default, ClickHouse applies the compression method, defined in [server settings](../operations/server_settings/settings.md#compression), to columns. You can also define the compression method for each individual column in the `CREATE TABLE` query. By default, ClickHouse applies the compression method, defined in [server settings](../operations/server_settings/settings.md#server-settings-compression), to columns. You can also define the compression method for each individual column in the `CREATE TABLE` query.
```sql ```sql
CREATE TABLE codec_example CREATE TABLE codec_example

View File

@ -444,7 +444,7 @@ JOIN的行为受 [join_use_nulls](../operations/settings/settings.md) 的影响
如果JOIN的key是 [Nullable](../data_types/nullable.md) 类型的字段,则其中至少一个存在 [NULL](syntax.md) 值的key不会被关联。 如果JOIN的key是 [Nullable](../data_types/nullable.md) 类型的字段,则其中至少一个存在 [NULL](syntax.md) 值的key不会被关联。
### WHERE 子句 ### WHERE 子句 {#select-where}
如果存在WHERE子句, 则在该子句中必须包含一个UInt8类型的表达式。 这个表达是通常是一个带有比较和逻辑的表达式。 如果存在WHERE子句, 则在该子句中必须包含一个UInt8类型的表达式。 这个表达是通常是一个带有比较和逻辑的表达式。
这个表达式将会在所有数据转换前用来过滤数据。 这个表达式将会在所有数据转换前用来过滤数据。

View File

@ -133,10 +133,13 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
split->addChannel(log); split->addChannel(log);
} }
bool is_tty = isatty(STDIN_FILENO) || isatty(STDERR_FILENO);
if (config.getBool("logger.console", false) if (config.getBool("logger.console", false)
|| (!config.hasProperty("logger.console") && !is_daemon && (isatty(STDIN_FILENO) || isatty(STDERR_FILENO)))) || (!config.hasProperty("logger.console") && !is_daemon && is_tty))
{ {
Poco::AutoPtr<DB::OwnFormattingChannel> log = new DB::OwnFormattingChannel(new OwnPatternFormatter(this), new Poco::ConsoleChannel); Poco::AutoPtr<OwnPatternFormatter> pf = new OwnPatternFormatter(this, OwnPatternFormatter::ADD_NOTHING, is_tty);
Poco::AutoPtr<DB::OwnFormattingChannel> log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel);
logger.warning("Logging " + log_level + " to console"); logger.warning("Logging " + log_level + " to console");
split->addChannel(log); split->addChannel(log);
} }

View File

@ -4,6 +4,7 @@
#include <optional> #include <optional>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Common/HashTable/Hash.h>
#include <Interpreters/InternalTextLogsQueue.h> #include <Interpreters/InternalTextLogsQueue.h>
#include <sys/time.h> #include <sys/time.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
@ -11,8 +12,64 @@
#include "Loggers.h" #include "Loggers.h"
OwnPatternFormatter::OwnPatternFormatter(const Loggers * loggers_, OwnPatternFormatter::Options options_) static const char * setColor(UInt64 num)
: Poco::PatternFormatter(""), loggers(loggers_), options(options_) {
/// ANSI escape sequences to set foreground font color in terminal.
static constexpr auto num_colors = 14;
static const char * colors[num_colors] =
{
/// Black on black is meaningless
"\033[40m\033[0;31m",
"\033[40m\033[0;32m",
"\033[40m\033[0;33m",
/// Low intense blue on black is too dark.
"\033[40m\033[0;35m",
"\033[40m\033[0;36m",
"\033[40m\033[0;37m",
"\033[40m\033[1;30m",
"\033[40m\033[1;31m",
"\033[40m\033[1;32m",
"\033[40m\033[1;33m",
"\033[40m\033[1;34m",
"\033[40m\033[1;35m",
"\033[40m\033[1;36m",
"\033[40m\033[1m", /// Not as white but just as high intense - for readability on white background.
};
return colors[num % num_colors];
}
static const char * setColorForLogPriority(int priority)
{
if (priority < 1 || priority > 8)
return "";
static const char * colors[] =
{
"",
"\033[40m\033[1;41m", /// Fatal
"\033[40m\033[0;41m", /// Critical
"\033[40m\033[1;31m", /// Error
"\033[40m\033[0;31m", /// Warning
"\033[40m\033[0;33m", /// Notice
"\033[40m\033[1m", /// Information
"", /// Debug
"\033[40m\033[1;30m", /// Trace
};
return colors[priority];
}
static const char * resetColor()
{
return "\033[0m";
}
OwnPatternFormatter::OwnPatternFormatter(const Loggers * loggers_, OwnPatternFormatter::Options options_, bool color_)
: Poco::PatternFormatter(""), loggers(loggers_), options(options_), color(color_)
{ {
} }
@ -48,17 +105,30 @@ void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext,
DB::writeChar('0' + ((msg_ext.time_microseconds / 1) % 10), wb); DB::writeChar('0' + ((msg_ext.time_microseconds / 1) % 10), wb);
writeCString(" [ ", wb); writeCString(" [ ", wb);
if (color)
writeCString(setColor(intHash64(msg_ext.thread_id)), wb);
DB::writeIntText(msg_ext.thread_id, wb); DB::writeIntText(msg_ext.thread_id, wb);
if (color)
writeCString(resetColor(), wb);
writeCString(" ] ", wb); writeCString(" ] ", wb);
/// We write query_id even in case when it is empty (no query context) /// We write query_id even in case when it is empty (no query context)
/// just to be convenient for various log parsers. /// just to be convenient for various log parsers.
writeCString("{", wb); writeCString("{", wb);
if (color)
writeCString(setColor(std::hash<std::string>()(msg_ext.query_id)), wb);
DB::writeString(msg_ext.query_id, wb); DB::writeString(msg_ext.query_id, wb);
if (color)
writeCString(resetColor(), wb);
writeCString("} ", wb); writeCString("} ", wb);
writeCString("<", wb); writeCString("<", wb);
DB::writeString(getPriorityName(static_cast<int>(msg.getPriority())), wb); int priority = static_cast<int>(msg.getPriority());
if (color)
writeCString(setColorForLogPriority(priority), wb);
DB::writeString(getPriorityName(priority), wb);
if (color)
writeCString(resetColor(), wb);
writeCString("> ", wb); writeCString("> ", wb);
DB::writeString(msg.getSource(), wb); DB::writeString(msg.getSource(), wb);
writeCString(": ", wb); writeCString(": ", wb);

View File

@ -31,7 +31,7 @@ public:
ADD_LAYER_TAG = 1 << 0 ADD_LAYER_TAG = 1 << 0
}; };
OwnPatternFormatter(const Loggers * loggers_, Options options_ = ADD_NOTHING); OwnPatternFormatter(const Loggers * loggers_, Options options_ = ADD_NOTHING, bool color_ = false);
void format(const Poco::Message & msg, std::string & text) override; void format(const Poco::Message & msg, std::string & text) override;
void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text); void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text);
@ -39,4 +39,5 @@ public:
private: private:
const Loggers * loggers; const Loggers * loggers;
Options options; Options options;
bool color;
}; };