Merge branch 'master' into complete_zk_api

This commit is contained in:
alesapin 2020-11-20 11:48:12 +03:00
commit 9920d77a93
36 changed files with 469 additions and 76 deletions

2
.gitmodules vendored
View File

@ -196,7 +196,7 @@
[submodule "contrib/rocksdb"]
path = contrib/rocksdb
url = https://github.com/facebook/rocksdb
branch = v6.11.4
branch = v6.14.5
[submodule "contrib/xz"]
path = contrib/xz
url = https://github.com/xz-mirror/xz

2
contrib/rocksdb vendored

@ -1 +1 @@
Subproject commit 963314ffd681596ef2738a95249fe4c1163ef87a
Subproject commit 35d8e36ef1b8e3e0759ca81215f855226a0a54bd

View File

@ -347,8 +347,9 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_builder.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_garbage.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_meta.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_reader.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_format.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_reader.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_sequential_reader.cc
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_writer.cc
${ROCKSDB_SOURCE_DIR}/db/builder.cc
${ROCKSDB_SOURCE_DIR}/db/c.cc
@ -394,6 +395,8 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/db/memtable_list.cc
${ROCKSDB_SOURCE_DIR}/db/merge_helper.cc
${ROCKSDB_SOURCE_DIR}/db/merge_operator.cc
${ROCKSDB_SOURCE_DIR}/db/output_validator.cc
${ROCKSDB_SOURCE_DIR}/db/periodic_work_scheduler.cc
${ROCKSDB_SOURCE_DIR}/db/range_del_aggregator.cc
${ROCKSDB_SOURCE_DIR}/db/range_tombstone_fragmenter.cc
${ROCKSDB_SOURCE_DIR}/db/repair.cc
@ -451,12 +454,12 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/monitoring/perf_level.cc
${ROCKSDB_SOURCE_DIR}/monitoring/persistent_stats_history.cc
${ROCKSDB_SOURCE_DIR}/monitoring/statistics.cc
${ROCKSDB_SOURCE_DIR}/monitoring/stats_dump_scheduler.cc
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_impl.cc
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_updater.cc
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util.cc
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util_debug.cc
${ROCKSDB_SOURCE_DIR}/options/cf_options.cc
${ROCKSDB_SOURCE_DIR}/options/configurable.cc
${ROCKSDB_SOURCE_DIR}/options/db_options.cc
${ROCKSDB_SOURCE_DIR}/options/options.cc
${ROCKSDB_SOURCE_DIR}/options/options_helper.cc
@ -507,6 +510,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/table/sst_file_dumper.cc
${ROCKSDB_SOURCE_DIR}/table/sst_file_reader.cc
${ROCKSDB_SOURCE_DIR}/table/sst_file_writer.cc
${ROCKSDB_SOURCE_DIR}/table/table_factory.cc
${ROCKSDB_SOURCE_DIR}/table/table_properties.cc
${ROCKSDB_SOURCE_DIR}/table/two_level_iterator.cc
${ROCKSDB_SOURCE_DIR}/test_util/sync_point.cc
@ -515,6 +519,7 @@ set(SOURCES
${ROCKSDB_SOURCE_DIR}/test_util/transaction_test_util.cc
${ROCKSDB_SOURCE_DIR}/tools/block_cache_analyzer/block_cache_trace_analyzer.cc
${ROCKSDB_SOURCE_DIR}/tools/dump/db_dump_tool.cc
${ROCKSDB_SOURCE_DIR}/tools/io_tracer_parser_tool.cc
${ROCKSDB_SOURCE_DIR}/tools/ldb_cmd.cc
${ROCKSDB_SOURCE_DIR}/tools/ldb_tool.cc
${ROCKSDB_SOURCE_DIR}/tools/sst_dump_tool.cc

View File

@ -86,20 +86,20 @@ while /bin/true; do
sleep 2
done &
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW DATABASES"
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary"
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "CREATE DATABASE test"
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-client --query "SHOW DATABASES"
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary"
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-client --query "CREATE DATABASE test"
kill_clickhouse
start_clickhouse
sleep 10
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW TABLES FROM datasets"
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW TABLES FROM test"
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits"
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW TABLES FROM test"
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-client --query "SHOW TABLES FROM datasets"
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-client --query "SHOW TABLES FROM test"
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits"
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-client --query "SHOW TABLES FROM test"
if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then
SKIP_LIST_OPT="--use-skip-list"
@ -109,7 +109,7 @@ fi
# more idiologically correct.
read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}"
LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
kill_clickhouse

View File

@ -2317,4 +2317,10 @@ Possible values:
Default value: `1`.
## output_format_tsv_null_representation {#output_format_tsv_null_representation}
Allows configurable `NULL` representation for [TSV](../../interfaces/formats.md#tabseparated) output format. The setting only controls output format and `\N` is the only supported `NULL` representation for TSV input format.
Default value: `\N`.
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->

View File

@ -0,0 +1,70 @@
# system.replicated_fetches {#system_tables-replicated_fetches}
Contains information about currently running background fetches.
Columns:
- `database` ([String](../../sql-reference/data-types/string.md)) — Name of the database.
- `table` ([String](../../sql-reference/data-types/string.md)) — Name of the table.
- `elapsed` ([Float64](../../sql-reference/data-types/float.md)) — The time elapsed (in seconds) since showing currently running background fetches started.
- `progress` ([Float64](../../sql-reference/data-types/float.md)) — The percentage of completed work from 0 to 1.
- `result_part_name` ([String](../../sql-reference/data-types/string.md)) — The name of the part that will be formed as the result of showing currently running background fetches.
- `result_part_path` ([String](../../sql-reference/data-types/string.md)) — Absolute path to the part that will be formed as the result of showing currently running background fetches.
- `partition_id` ([String](../../sql-reference/data-types/string.md)) — ID of the partition.
- `total_size_bytes_compressed` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The total size (in bytes) of the compressed data in the result part.
- `bytes_read_compressed` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of compressed bytes read from the result part.
- `source_replica_path` ([String](../../sql-reference/data-types/string.md)) — Absolute path to the source replica.
- `source_replica_hostname` ([String](../../sql-reference/data-types/string.md)) — Hostname of the source replica.
- `source_replica_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — Port number of the source replica.
- `interserver_scheme` ([String](../../sql-reference/data-types/string.md)) — Name of the interserver scheme.
- `URI` ([String](../../sql-reference/data-types/string.md)) — Uniform resource identifier.
- `to_detached` ([UInt8](../../sql-reference/data-types/int-uint.md)) — The flag indicates whether the currently running background fetch is being performed using the `TO DETACHED` expression.
- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Thread identifier.
**Example**
``` sql
SELECT * FROM system.replicated_fetches LIMIT 1 FORMAT Vertical;
```
``` text
Row 1:
──────
database: default
table: t
elapsed: 7.243039876
progress: 0.41832135995612835
result_part_name: all_0_0_0
result_part_path: /var/lib/clickhouse/store/700/70080a04-b2de-4adf-9fa5-9ea210e81766/all_0_0_0/
partition_id: all
total_size_bytes_compressed: 1052783726
bytes_read_compressed: 440401920
source_replica_path: /clickhouse/test/t/replicas/1
source_replica_hostname: node1
source_replica_port: 9009
interserver_scheme: http
URI: http://node1:9009/?endpoint=DataPartsExchange%3A%2Fclickhouse%2Ftest%2Ft%2Freplicas%2F1&part=all_0_0_0&client_protocol_version=4&compress=false
to_detached: 0
thread_id: 54
```
**See Also**
- [Managing ReplicatedMergeTree Tables](../../sql-reference/statements/system/#query-language-system-replicated)
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/replicated_fetches) <!--hide-->

View File

@ -115,7 +115,21 @@ Returns the “first significant subdomain”. This is a non-standard concept sp
Returns the part of the domain that includes top-level subdomains up to the “first significant subdomain” (see the explanation above).
For example, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'yandex.com.tr'`.
For example:
- `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'yandex.com.tr'`.
- `cutToFirstSignificantSubdomain('www.tr') = 'tr'`.
- `cutToFirstSignificantSubdomain('tr') = ''`.
### cutToFirstSignificantSubdomainWithWWW {#cuttofirstsignificantsubdomainwithwww}
Returns the part of the domain that includes top-level subdomains up to the “first significant subdomain”, without stripping "www".
For example:
- `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'yandex.com.tr'`.
- `cutToFirstSignificantSubdomain('www.tr') = 'www.tr'`.
- `cutToFirstSignificantSubdomain('tr') = ''`.
### port(URL\[, default_port = 0\]) {#port}

View File

@ -27,9 +27,9 @@ It is applicable when selecting data from tables that use the [MergeTree](../../
### Drawbacks {#drawbacks}
Queries that use `FINAL` are executed not as fast as similar queries that dont, because:
Queries that use `FINAL` are executed slightly slower than similar queries that dont, because:
- Query is executed in a single thread and data is merged during query execution.
- Data is merged during query execution.
- Queries with `FINAL` read primary key columns in addition to the columns specified in the query.
**In most cases, avoid using `FINAL`.** The common approach is to use different queries that assume the background processes of the `MergeTree` engine havet happened yet and deal with it by applying aggregation (for example, to discard duplicates). {## TODO: examples ##}

View File

@ -2187,4 +2187,10 @@ SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x);
Значение по умолчанию: `1`.
## output_format_tsv_null_representation {#output_format_tsv_null_representation}
Позволяет настраивать представление `NULL` для формата выходных данных [TSV](../../interfaces/formats.md#tabseparated). Настройка управляет форматом выходных данных, `\N` является единственным поддерживаемым представлением для формата входных данных TSV.
Значение по умолчанию: `\N`.
[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) <!--hide-->

View File

@ -0,0 +1,70 @@
# system.replicated_fetches {#system_tables-replicated_fetches}
Содержит информацию о выполняемых в данный момент фоновых операциях скачивания кусков данных с других реплик.
Столбцы:
- `database` ([String](../../sql-reference/data-types/string.md)) — имя базы данных.
- `table` ([String](../../sql-reference/data-types/string.md)) — имя таблицы.
- `elapsed` ([Float64](../../sql-reference/data-types/float.md)) — время, прошедшее от момента начала скачивания куска, в секундах.
- `progress` ([Float64](../../sql-reference/data-types/float.md)) — доля выполненной работы от 0 до 1.
- `result_part_name` ([String](../../sql-reference/data-types/string.md)) — имя скачиваемого куска.
- `result_part_path` ([String](../../sql-reference/data-types/string.md)) — абсолютный путь к скачиваемому куску.
- `partition_id` ([String](../../sql-reference/data-types/string.md)) — идентификатор партиции.
- `total_size_bytes_compressed` ([UInt64](../../sql-reference/data-types/int-uint.md)) — общий размер сжатой информации в скачиваемом куске в байтах.
- `bytes_read_compressed` ([UInt64](../../sql-reference/data-types/int-uint.md)) — размер сжатой информации, считанной из скачиваемого куска, в байтах.
- `source_replica_path` ([String](../../sql-reference/data-types/string.md)) — абсолютный путь к исходной реплике.
- `source_replica_hostname` ([String](../../sql-reference/data-types/string.md)) — имя хоста исходной реплики.
- `source_replica_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — номер порта исходной реплики.
- `interserver_scheme` ([String](../../sql-reference/data-types/string.md)) — имя межсерверной схемы.
- `URI` ([String](../../sql-reference/data-types/string.md)) — универсальный идентификатор ресурса.
- `to_detached` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, указывающий на использование выражения `TO DETACHED` в текущих фоновых операциях.
- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — идентификатор потока.
**Пример**
``` sql
SELECT * FROM system.replicated_fetches LIMIT 1 FORMAT Vertical;
```
``` text
Row 1:
──────
database: default
table: t
elapsed: 7.243039876
progress: 0.41832135995612835
result_part_name: all_0_0_0
result_part_path: /var/lib/clickhouse/store/700/70080a04-b2de-4adf-9fa5-9ea210e81766/all_0_0_0/
partition_id: all
total_size_bytes_compressed: 1052783726
bytes_read_compressed: 440401920
source_replica_path: /clickhouse/test/t/replicas/1
source_replica_hostname: node1
source_replica_port: 9009
interserver_scheme: http
URI: http://node1:9009/?endpoint=DataPartsExchange%3A%2Fclickhouse%2Ftest%2Ft%2Freplicas%2F1&part=all_0_0_0&client_protocol_version=4&compress=false
to_detached: 0
thread_id: 54
```
**Смотрите также**
- [Управление таблицами ReplicatedMergeTree](../../sql-reference/statements/system/#query-language-system-replicated)
[Оригинальная статья](https://clickhouse.tech/docs/en/operations/system_tables/replicated_fetches) <!--hide-->

View File

@ -27,9 +27,9 @@ toc_title: FROM
### Недостатки {#drawbacks}
Запросы, которые используют `FINAL` выполняются не так быстро, как аналогичные запросы без него, потому что:
Запросы, которые используют `FINAL` выполняются немного медленее, чем аналогичные запросы без него, потому что:
- Запрос выполняется в одном потоке, и данные мёржатся во время выполнения запроса.
- Данные мёржатся во время выполнения запроса.
- Запросы с модификатором `FINAL` читают столбцы первичного ключа в дополнение к столбцам, используемым в запросе.
**В большинстве случаев избегайте использования `FINAL`.** Общий подход заключается в использовании агрегирующих запросов, которые предполагают, что фоновые процессы движков семейства `MergeTree` ещё не случились (например, сами отбрасывают дубликаты). {## TODO: examples ##}

View File

@ -441,6 +441,7 @@ class IColumn;
M(Bool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \
\
M(Bool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \
M(Bool, output_format_json_array_of_rows, false, "Output a JSON array of all rows in JSONEachRow(Compact) format.", 0) \
\
M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \
M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \
@ -517,7 +518,7 @@ struct Settings : public BaseSettings<SettingsTraits>
};
/*
* User-specified file format settings for File and ULR engines.
* User-specified file format settings for File and URL engines.
*/
DECLARE_SETTINGS_TRAITS(FormatFactorySettingsTraits, FORMAT_FACTORY_SETTINGS)

View File

@ -78,6 +78,7 @@ FormatSettings getFormatSettings(const Context & context,
format_settings.import_nested_json = settings.input_format_import_nested_json;
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
format_settings.json.array_of_rows = settings.output_format_json_array_of_rows;
format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes;
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;

View File

@ -86,6 +86,7 @@ struct FormatSettings
struct
{
bool array_of_rows = false;
bool quote_64bit_integers = true;
bool quote_denormals = true;
bool escape_forward_slashes = true;

View File

@ -6,6 +6,7 @@
namespace DB
{
template <bool without_www>
struct CutToFirstSignificantSubdomain
{
static size_t getReserveLengthForElement() { return 15; }
@ -18,7 +19,7 @@ struct CutToFirstSignificantSubdomain
Pos tmp_data;
size_t tmp_length;
Pos domain_end;
ExtractFirstSignificantSubdomain::execute(data, size, tmp_data, tmp_length, &domain_end);
ExtractFirstSignificantSubdomain<without_www>::execute(data, size, tmp_data, tmp_length, &domain_end);
if (tmp_length == 0)
return;
@ -29,11 +30,15 @@ struct CutToFirstSignificantSubdomain
};
struct NameCutToFirstSignificantSubdomain { static constexpr auto name = "cutToFirstSignificantSubdomain"; };
using FunctionCutToFirstSignificantSubdomain = FunctionStringToString<ExtractSubstringImpl<CutToFirstSignificantSubdomain>, NameCutToFirstSignificantSubdomain>;
using FunctionCutToFirstSignificantSubdomain = FunctionStringToString<ExtractSubstringImpl<CutToFirstSignificantSubdomain<true>>, NameCutToFirstSignificantSubdomain>;
struct NameCutToFirstSignificantSubdomainWithWWW { static constexpr auto name = "cutToFirstSignificantSubdomainWithWWW"; };
using FunctionCutToFirstSignificantSubdomainWithWWW = FunctionStringToString<ExtractSubstringImpl<CutToFirstSignificantSubdomain<false>>, NameCutToFirstSignificantSubdomainWithWWW>;
void registerFunctionCutToFirstSignificantSubdomain(FunctionFactory & factory)
{
factory.registerFunction<FunctionCutToFirstSignificantSubdomain>();
factory.registerFunction<FunctionCutToFirstSignificantSubdomainWithWWW>();
}
}

View File

@ -7,7 +7,7 @@ namespace DB
{
struct NameFirstSignificantSubdomain { static constexpr auto name = "firstSignificantSubdomain"; };
using FunctionFirstSignificantSubdomain = FunctionStringToString<ExtractSubstringImpl<ExtractFirstSignificantSubdomain>, NameFirstSignificantSubdomain>;
using FunctionFirstSignificantSubdomain = FunctionStringToString<ExtractSubstringImpl<ExtractFirstSignificantSubdomain<true>>, NameFirstSignificantSubdomain>;
void registerFunctionFirstSignificantSubdomain(FunctionFactory & factory)
{

View File

@ -7,6 +7,7 @@
namespace DB
{
template <bool without_www>
struct ExtractFirstSignificantSubdomain
{
static size_t getReserveLengthForElement() { return 10; }
@ -18,7 +19,7 @@ struct ExtractFirstSignificantSubdomain
Pos tmp;
size_t domain_length;
ExtractDomain<true>::execute(data, size, tmp, domain_length);
ExtractDomain<without_www>::execute(data, size, tmp, domain_length);
if (domain_length == 0)
return;

View File

@ -12,9 +12,9 @@ JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSettings & settings_,
bool yield_strings_)
: IRowOutputFormat(header_, out_, params_), settings(settings_), yield_strings(yield_strings_)
const FormatSettings & settings_)
: IRowOutputFormat(header_, out_, params_),
settings(settings_)
{
const auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
@ -33,7 +33,7 @@ void JSONEachRowRowOutputFormat::writeField(const IColumn & column, const IDataT
writeString(fields[field_number], out);
writeChar(':', out);
if (yield_strings)
if (settings.json.serialize_as_strings)
{
WriteBufferFromOwnString buf;
@ -61,29 +61,94 @@ void JSONEachRowRowOutputFormat::writeRowStartDelimiter()
void JSONEachRowRowOutputFormat::writeRowEndDelimiter()
{
writeCString("}\n", out);
// Why do we need this weird `if`?
//
// The reason is the formatRow function that is broken with respect to
// row-between delimiters. It should not write them, but it does, and then
// hacks around it by having a special formatRowNoNewline version, which, as
// you guessed, removes the newline from the end of row. But the row-between
// delimiter goes into a second row, so it turns out to be in the beginning
// of the line, and the removal doesn't work. There is also a second bug --
// the row-between delimiter in this format is written incorrectly. In fact,
// it is not written at all, and the newline is written in a row-end
// delimiter ("}\n" instead of the correct "}"). With these two bugs
// combined, the test 01420_format_row works perfectly.
//
// A proper implementation of formatRow would use IRowOutputFormat directly,
// and not write row-between delimiters, instead of using IOutputFormat
// processor and its crutch row callback. This would require exposing
// IRowOutputFormat, which we don't do now, but which can be generally useful
// for other cases such as parallel formatting, that also require a control
// flow different from the usual IOutputFormat.
//
// I just don't have time or energy to redo all of this, but I need to
// support JSON array output here, which requires proper ",\n" row-between
// delimiters. For compatibility, I preserve the bug in case of non-array
// output.
if (settings.json.array_of_rows)
{
writeCString("}", out);
}
else
{
writeCString("}\n", out);
}
field_number = 0;
}
void JSONEachRowRowOutputFormat::writeRowBetweenDelimiter()
{
// We preserve an existing bug here for compatibility. See the comment above.
if (settings.json.array_of_rows)
{
writeCString(",\n", out);
}
}
void JSONEachRowRowOutputFormat::writePrefix()
{
if (settings.json.array_of_rows)
{
writeCString("[\n", out);
}
}
void JSONEachRowRowOutputFormat::writeSuffix()
{
if (settings.json.array_of_rows)
{
writeCString("\n]\n", out);
}
}
void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("JSONEachRow", [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
const FormatSettings & _format_settings)
{
return std::make_shared<JSONEachRowRowOutputFormat>(buf, sample, params, format_settings, false);
FormatSettings settings = _format_settings;
settings.json.serialize_as_strings = false;
return std::make_shared<JSONEachRowRowOutputFormat>(buf, sample, params,
settings);
});
factory.registerOutputFormatProcessor("JSONStringsEachRow", [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
const FormatSettings & _format_settings)
{
return std::make_shared<JSONEachRowRowOutputFormat>(buf, sample, params, format_settings, true);
FormatSettings settings = _format_settings;
settings.json.serialize_as_strings = true;
return std::make_shared<JSONEachRowRowOutputFormat>(buf, sample, params,
settings);
});
}

View File

@ -19,8 +19,7 @@ public:
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSettings & settings_,
bool yield_strings_);
const FormatSettings & settings_);
String getName() const override { return "JSONEachRowRowOutputFormat"; }
@ -28,6 +27,9 @@ public:
void writeFieldDelimiter() override;
void writeRowStartDelimiter() override;
void writeRowEndDelimiter() override;
void writeRowBetweenDelimiter() override;
void writePrefix() override;
void writeSuffix() override;
protected:
/// No totals and extremes.
@ -40,9 +42,6 @@ private:
Names fields;
FormatSettings settings;
protected:
bool yield_strings;
};
}

View File

@ -34,18 +34,24 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
const FormatSettings & _format_settings)
{
return std::make_shared<JSONEachRowWithProgressRowOutputFormat>(buf, sample, params, format_settings, false);
FormatSettings settings = _format_settings;
settings.json.serialize_as_strings = false;
return std::make_shared<JSONEachRowWithProgressRowOutputFormat>(buf,
sample, params, settings);
});
factory.registerOutputFormatProcessor("JSONStringsEachRowWithProgress", [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
const FormatSettings & _format_settings)
{
return std::make_shared<JSONEachRowWithProgressRowOutputFormat>(buf, sample, params, format_settings, true);
FormatSettings settings = _format_settings;
settings.json.serialize_as_strings = true;
return std::make_shared<JSONEachRowWithProgressRowOutputFormat>(buf,
sample, params, settings);
});
}

View File

@ -44,6 +44,29 @@ void IMergeTreeDataPartWriter::next()
index_offset = next_index_offset;
}
bool IMergeTreeDataPartWriter::adjustLastUnfinishedMark(size_t new_block_index_granularity)
{
/// If amount of rest rows in the last granule more then granularity of the new block
/// than finish it.
if (!index_granularity.empty() && index_offset > new_block_index_granularity)
{
size_t already_written_rows_in_last_granule = index_granularity.getLastMarkRows() - index_offset;
/// We can still write some rows to the last granule
if (already_written_rows_in_last_granule < new_block_index_granularity)
{
index_granularity.setLastMarkRows(new_block_index_granularity);
index_offset = new_block_index_granularity - already_written_rows_in_last_granule;
}
else /// Our last granule is already full, let's start the new one
{
index_granularity.setLastMarkRows(already_written_rows_in_last_granule);
index_offset = 0;
}
return true;
}
return false;
}
IMergeTreeDataPartWriter::~IMergeTreeDataPartWriter() = default;
}

View File

@ -62,28 +62,41 @@ public:
protected:
size_t getCurrentMark() const { return current_mark; }
size_t getIndexOffset() const { return index_offset; }
/// Finishes our current unfinished mark if we have already written more rows for it
/// than granularity in the new block. Return true if last mark actually was adjusted.
/// Example:
/// __|________|___. <- previous block with granularity 8 and last unfinished mark with 3 rows
/// new_block_index_granularity = 2, so
/// __|________|___|__|__|__|
/// ^ finish last unfinished mark, new marks will have granularity 2
bool adjustLastUnfinishedMark(size_t new_block_index_granularity);
using SerializationState = IDataType::SerializeBinaryBulkStatePtr;
using SerializationStates = std::unordered_map<String, SerializationState>;
MergeTreeData::DataPartPtr data_part;
const MergeTreeData & storage;
StorageMetadataPtr metadata_snapshot;
NamesAndTypesList columns_list;
MergeTreeIndices skip_indices;
const StorageMetadataPtr metadata_snapshot;
const NamesAndTypesList columns_list;
const MergeTreeIndices skip_indices;
MergeTreeIndexGranularity index_granularity;
MergeTreeWriterSettings settings;
bool with_final_mark;
const MergeTreeWriterSettings settings;
const bool with_final_mark;
size_t next_mark = 0;
size_t next_index_offset = 0;
/// When we were writing fresh block granularity of the last mark was adjusted
/// See adjustLastUnfinishedMark
bool last_granule_was_adjusted = false;
MutableColumns index_columns;
private:
/// Data is already written up to this mark.
size_t current_mark = 0;
/// The offset to the first row of the block for which you want to write the index.
/// Or how many rows we have to write for this last unfinished mark.
size_t index_offset = 0;
};

View File

@ -184,7 +184,8 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices()
default_codec, settings.max_compress_block_size,
0, settings.aio_threshold));
skip_indices_aggregators.push_back(index_helper->createIndexAggregator());
skip_index_filling.push_back(0);
marks_in_skip_index_aggregator.push_back(0);
rows_in_skip_index_aggregator_last_mark.push_back(0);
}
skip_indices_initialized = true;
@ -256,9 +257,11 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
skip_index_current_data_mark = skip_index_data_mark;
while (prev_pos < rows)
{
bool new_block_started = prev_pos == 0;
UInt64 limit = 0;
size_t current_index_offset = getIndexOffset();
if (prev_pos == 0 && current_index_offset != 0)
/// We start new block, but have an offset from previous one
if (new_block_started && current_index_offset != 0)
{
limit = current_index_offset;
}
@ -270,10 +273,15 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
else
{
limit = index_granularity.getMarkRows(skip_index_current_data_mark);
/// We just started new block serialization but last unfinished mark was shrinked to it's current_size
/// it may happen that we have already aggregated current_size of rows of more for skip_index, but not flushed it to disk
/// because previous granule size was bigger. So do it here.
if (new_block_started && last_granule_was_adjusted && rows_in_skip_index_aggregator_last_mark[i] >= limit)
accountMarkForSkipIdxAndFlushIfNeeded(i);
if (skip_indices_aggregators[i]->empty())
{
skip_indices_aggregators[i] = index_helper->createIndexAggregator();
skip_index_filling[i] = 0;
if (stream.compressed.offset() >= settings.min_compress_block_size)
stream.compressed.next();
@ -285,24 +293,19 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
if (settings.can_use_adaptive_granularity)
writeIntBinary(1UL, stream.marks);
}
/// this mark is aggregated, go to the next one
skip_index_current_data_mark++;
}
size_t pos = prev_pos;
skip_indices_aggregators[i]->update(skip_indexes_block, &pos, limit);
rows_in_skip_index_aggregator_last_mark[i] = (pos - prev_pos);
if (pos == prev_pos + limit)
{
++skip_index_filling[i];
/// write index if it is filled
if (skip_index_filling[i] == index_helper->index.granularity)
{
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed);
skip_index_filling[i] = 0;
}
}
/// We just aggregated all rows in current mark, add new mark to skip_index marks counter
/// and flush on disk if we already aggregated required amount of marks.
if (rows_in_skip_index_aggregator_last_mark[i] == limit)
accountMarkForSkipIdxAndFlushIfNeeded(i);
prev_pos = pos;
}
}
@ -360,7 +363,21 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(
skip_indices_streams.clear();
skip_indices_aggregators.clear();
skip_index_filling.clear();
marks_in_skip_index_aggregator.clear();
rows_in_skip_index_aggregator_last_mark.clear();
}
void MergeTreeDataPartWriterOnDisk::accountMarkForSkipIdxAndFlushIfNeeded(size_t skip_index_pos)
{
++marks_in_skip_index_aggregator[skip_index_pos];
/// write index if it is filled
if (marks_in_skip_index_aggregator[skip_index_pos] == skip_indices[skip_index_pos]->index.granularity)
{
skip_indices_aggregators[skip_index_pos]->getGranuleAndReset()->serializeBinary(skip_indices_streams[skip_index_pos]->compressed);
marks_in_skip_index_aggregator[skip_index_pos] = 0;
rows_in_skip_index_aggregator_last_mark[skip_index_pos] = 0;
}
}
}

View File

@ -97,8 +97,7 @@ protected:
const String marks_file_extension;
CompressionCodecPtr default_codec;
bool compute_granularity;
bool need_finish_last_granule;
const bool compute_granularity;
/// Number of marsk in data from which skip indices have to start
/// aggregation. I.e. it's data mark number, not skip indices mark.
@ -106,7 +105,10 @@ protected:
std::vector<StreamPtr> skip_indices_streams;
MergeTreeIndexAggregators skip_indices_aggregators;
std::vector<size_t> skip_index_filling;
/// Amount of marks currently serialized in skip index aggregator
std::vector<size_t> marks_in_skip_index_aggregator;
/// Amount of rows currently serialized in skip index aggregator for last mark
std::vector<size_t> rows_in_skip_index_aggregator_last_mark;
std::unique_ptr<WriteBufferFromFileBase> index_file_stream;
std::unique_ptr<HashingWriteBuffer> index_stream;
@ -125,6 +127,11 @@ protected:
private:
/// Index is already serialized up to this mark.
size_t index_mark = 0;
/// Increment corresponding marks_in_skip_index_aggregator[skip_index_pos]
/// value and flush skip_indices_streams[skip_index_pos] to disk if we have
/// aggregated enough marks
void accountMarkForSkipIdxAndFlushIfNeeded(size_t skip_index_pos);
};
}

View File

@ -95,6 +95,9 @@ void MergeTreeDataPartWriterWide::write(const Block & block,
if (compute_granularity)
{
size_t index_granularity_for_block = computeIndexGranularity(block);
/// Finish last unfinished mark rows it it's required
last_granule_was_adjusted = adjustLastUnfinishedMark(index_granularity_for_block);
/// Fill index granularity with granules of new size
fillIndexGranularity(index_granularity_for_block, block.rows());
}

View File

@ -55,6 +55,17 @@ void MergeTreeIndexGranularity::addRowsToLastMark(size_t rows_count)
marks_rows_partial_sums.back() += rows_count;
}
void MergeTreeIndexGranularity::setLastMarkRows(size_t rows_count)
{
if (marks_rows_partial_sums.empty())
marks_rows_partial_sums.push_back(rows_count);
else
{
marks_rows_partial_sums.back() -= getLastMarkRows();
marks_rows_partial_sums.back() += rows_count;
}
}
void MergeTreeIndexGranularity::popMark()
{
if (!marks_rows_partial_sums.empty())

View File

@ -98,6 +98,10 @@ public:
/// Extends last mark by rows_count.
void addRowsToLastMark(size_t rows_count);
/// Set amount of rows to last mark
/// (add new mark if new have nothing)
void setLastMarkRows(size_t rows_count);
/// Drops last mark if any exists.
void popMark();

View File

@ -1048,13 +1048,11 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
}
/// Check that fetches pool is not overloaded
if (entry.type == LogEntry::GET_PART)
if (entry.type == LogEntry::GET_PART && !storage.canExecuteFetch(entry, out_postpone_reason))
{
if (!storage.canExecuteFetch(entry, out_postpone_reason))
{
LOG_TRACE(log, out_postpone_reason);
return false;
}
/// Don't print log message about this, because we can have a lot of fetches,
/// for example during replica recovery.
return false;
}
if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::MUTATE_PART)

View File

@ -3,6 +3,7 @@
<optimize_aggregation_in_order>1</optimize_aggregation_in_order>
<optimize_read_in_order>1</optimize_read_in_order>
<max_partitions_per_insert_block>2000</max_partitions_per_insert_block>
<max_insert_block_size>10000000</max_insert_block_size>
<max_threads>8</max_threads>
</settings>
@ -16,8 +17,10 @@
</substitution>
</substitutions>
<create_query>CREATE TABLE mt_{parts}_parts(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % {parts}</create_query>
<fill_query>INSERT INTO mt_{parts}_parts SELECT number, rand() % 10000, rand() FROM numbers_mt(100000000)</fill_query>
<create_query>CREATE TABLE mt_{parts}_parts(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % {parts} SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000</create_query>
<fill_query>SYSTEM STOP MERGES mt_{parts}_parts</fill_query>
<fill_query>INSERT INTO mt_{parts}_parts SELECT number, rand() % 10000, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000</fill_query>
<fill_query>SYSTEM START MERGES mt_{parts}_parts</fill_query>
<fill_query>OPTIMIZE TABLE mt_{parts}_parts FINAL</fill_query>
<query>SELECT val2 FROM mt_{parts}_parts ORDER BY val1 LIMIT 1000000 FORMAT Null</query>

View File

@ -78,6 +78,15 @@ example.com
example.com
example.com
example.com
com
====CUT TO FIRST SIGNIFICANT SUBDOMAIN WITH WWW====
www.com
example.com
example.com
example.com
example.com
====CUT WWW====
http://example.com
http://example.com:1234

View File

@ -86,6 +86,16 @@ SELECT cutToFirstSignificantSubdomain('http://paul@www.example.com/a/b/c?a=b#d=f
SELECT cutToFirstSignificantSubdomain('//paul@www.example.com/a/b/c?a=b#d=f');
SELECT cutToFirstSignificantSubdomain('www.example.com');
SELECT cutToFirstSignificantSubdomain('example.com');
SELECT cutToFirstSignificantSubdomain('www.com');
SELECT cutToFirstSignificantSubdomain('com');
SELECT '====CUT TO FIRST SIGNIFICANT SUBDOMAIN WITH WWW====';
SELECT cutToFirstSignificantSubdomainWithWWW('http://com');
SELECT cutToFirstSignificantSubdomainWithWWW('http://www.com');
SELECT cutToFirstSignificantSubdomainWithWWW('http://www.example.com');
SELECT cutToFirstSignificantSubdomainWithWWW('http://www.foo.example.com');
SELECT cutToFirstSignificantSubdomainWithWWW('http://www.example.com:1');
SELECT cutToFirstSignificantSubdomainWithWWW('http://www.example.com/');
SELECT '====CUT WWW====';
SELECT cutWWW('http://www.example.com');

View File

@ -119,7 +119,7 @@ INSERT INTO mt_without_pk (d, x, y, z, `n.Age`, `n.Name`) VALUES (toDate('2018-1
SELECT COUNT(*) FROM mt_without_pk WHERE x > toDateTime('2018-10-01 23:57:57');
SELECT sum(marks) FROM system.parts WHERE table = 'mt_without_pk' AND active=1;
SELECT sum(marks) FROM system.parts WHERE table = 'mt_without_pk' AND active=1 AND database=currentDatabase();
INSERT INTO mt_without_pk (d, x, y, z, `n.Age`, `n.Name`) VALUES (toDate('2018-10-01'), toDateTime('2018-10-01 07:57:57'), [4, 4, 4], 14, [111, 222], ['Lui', 'Dave']), (toDate('2018-10-01'), toDateTime('2018-10-01 08:57:57'), [5, 5, 5], 15, [333, 444], ['John', 'Mike']), (toDate('2018-10-01'), toDateTime('2018-10-01 09:57:57'), [6, 6, 6], 16, [555, 666, 777], ['Alex', 'Jim', 'Tom']);
@ -127,7 +127,7 @@ OPTIMIZE TABLE mt_without_pk FINAL;
SELECT COUNT(*) FROM mt_without_pk WHERE x > toDateTime('2018-10-01 23:57:57');
SELECT sum(marks) FROM system.parts WHERE table = 'mt_without_pk' AND active=1;
SELECT sum(marks) FROM system.parts WHERE table = 'mt_without_pk' AND active=1 AND database=currentDatabase();
DROP TABLE IF EXISTS mt_without_pk;
@ -149,7 +149,7 @@ INSERT INTO mt_with_small_granularity (d, x, y, z, `n.Age`, `n.Name`) VALUES (to
SELECT COUNT(*) FROM mt_with_small_granularity WHERE x > toDateTime('2018-10-01 23:57:57');
SELECT sum(marks) FROM system.parts WHERE table = 'mt_with_small_granularity' AND active=1;
SELECT sum(marks) FROM system.parts WHERE table = 'mt_with_small_granularity' AND active=1 AND database=currentDatabase();
INSERT INTO mt_with_small_granularity (d, x, y, z, `n.Age`, `n.Name`) VALUES (toDate('2018-10-01'), toDateTime('2018-10-01 07:57:57'), [4, 4, 4], 14, [111, 222], ['Lui', 'Dave']), (toDate('2018-10-01'), toDateTime('2018-10-01 08:57:57'), [5, 5, 5], 15, [333, 444], ['John', 'Mike']), (toDate('2018-10-01'), toDateTime('2018-10-01 09:57:57'), [6, 6, 6], 16, [555, 666, 777], ['Alex', 'Jim', 'Tom']);
@ -157,6 +157,6 @@ OPTIMIZE TABLE mt_with_small_granularity FINAL;
SELECT COUNT(*) FROM mt_with_small_granularity WHERE x > toDateTime('2018-10-01 23:57:57');
SELECT sum(marks) FROM system.parts WHERE table = 'mt_with_small_granularity' AND active=1;
SELECT sum(marks) FROM system.parts WHERE table = 'mt_with_small_granularity' AND active=1 AND database=currentDatabase();
DROP TABLE IF EXISTS mt_with_small_granularity;

View File

@ -0,0 +1,11 @@
[
{"a":"0","b":"0"},
{"a":"1","b":"2"},
{"a":"2","b":"4"}
]
[
{"number":"0"}
]
[
]

View File

@ -0,0 +1,4 @@
set output_format_json_array_of_rows = 1;
select number a, number * 2 b from numbers(3) format JSONEachRow;
select * from numbers(1) format JSONEachRow;
select * from numbers(1) where null format JSONEachRow;

View File

@ -0,0 +1,2 @@
849
102400

View File

@ -0,0 +1,28 @@
DROP TABLE IF EXISTS adaptive_table;
--- If granularity of consequent blocks differs a lot, then adaptive
--- granularity will adjust amout of marks correctly. Data for test empirically
--- derived, it's quite hard to get good parameters.
CREATE TABLE adaptive_table(
key UInt64,
value String
) ENGINE MergeTree()
ORDER BY key
SETTINGS index_granularity_bytes=1048576, min_bytes_for_wide_part = 0, enable_vertical_merge_algorithm = 0;
SET max_block_size=900;
-- There are about 900 marks for our settings.
INSERT INTO adaptive_table SELECT number, if(number > 700, randomPrintableASCII(102400), randomPrintableASCII(1)) FROM numbers(10000);
OPTIMIZE TABLE adaptive_table FINAL;
SELECT marks FROM system.parts WHERE table = 'adaptive_table' and database=currentDatabase() and active;
-- If we have computed granularity incorrectly than we will exceed this limit.
SET max_memory_usage='30M';
SELECT max(length(value)) FROM adaptive_table;
DROP TABLE IF EXISTS adaptive_table;