mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-09-21 01:00:48 +00:00
Merge branch 'master' into joins
This commit is contained in:
commit
ba91920b19
19
.github/PULL_REQUEST_TEMPLATE.md
vendored
19
.github/PULL_REQUEST_TEMPLATE.md
vendored
@ -1 +1,20 @@
|
||||
I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla/?lang=en
|
||||
|
||||
For changelog. Remove if this is non-significant change.
|
||||
|
||||
Category (leave one):
|
||||
- New Feature
|
||||
- Bug Fix
|
||||
- Improvement
|
||||
- Performance Improvement
|
||||
- Backward Incompatible Change
|
||||
- Build/Testing/Packaging Improvement
|
||||
- Other
|
||||
|
||||
Short description (up to few sentences):
|
||||
|
||||
...
|
||||
|
||||
Detailed description (optional):
|
||||
|
||||
...
|
||||
|
@ -1,4 +1 @@
|
||||
* Настройка `enable_optimize_predicate_expression` выключена по-умолчанию.
|
||||
|
||||
### Улучшения:
|
||||
* Файлы *-preprocessed.xml записываются в директорию с данными (/var/lib/clickhouse/preprocessed_configs). Для /etc/clickhouse-server больше не нужен +w для пользователя clickhouse. Для удобства создан симлинк /var/lib/clickhouse/preprocessed_configs -> /etc/clickhouse-server/preprocessed
|
||||
|
@ -926,7 +926,7 @@ This release contains bug fixes for the previous release 1.1.54310:
|
||||
### New features:
|
||||
|
||||
* Custom partitioning key for the MergeTree family of table engines.
|
||||
* [ Kafka](https://clickhouse.yandex/docs/en/single/index.html#document-table_engines/kafka) table engine.
|
||||
* [Kafka](https://clickhouse.yandex/docs/en/operations/table_engines/kafka/) table engine.
|
||||
* Added support for loading [CatBoost](https://catboost.yandex/) models and applying them to data stored in ClickHouse.
|
||||
* Added support for time zones with non-integer offsets from UTC.
|
||||
* Added support for arithmetic operations with time intervals.
|
||||
|
@ -1,3 +1,94 @@
|
||||
## ClickHouse release 18.16.0, 2018-12-14
|
||||
|
||||
### Новые возможности:
|
||||
|
||||
* Вычисление `DEFAULT` выражений для отсутствующих полей при загрузке данных в полуструктурированных форматах (`JSONEachRow`, `TSKV`).
|
||||
* Для запроса `ALTER TABLE` добавлено действие `MODIFY ORDER BY` для изменения ключа сортировки при одновременном добавлении или удалении столбца таблицы. Это полезно для таблиц семейства `MergeTree`, выполняющих дополнительную работу при слияниях, согласно этому ключу сортировки, как например, `SummingMergeTree`, `AggregatingMergeTree` и т. п.
|
||||
* Для таблиц семейства `MergeTree` появилась возможность указать различный ключ сортировки (`ORDER BY`) и индекс (`PRIMARY KEY`). Ключ сортировки может быть длиннее, чем индекс.
|
||||
* Добавлена табличная функция `hdfs` и движок таблиц `HDFS` для импорта и экспорта данных в HDFS.
|
||||
* Добавлены функции для работы с base64: `base64Encode`, `base64Decode`, `tryBase64Decode`.
|
||||
* Для агрегатной функции `uniqCombined` появилась возможность настраивать точность работы с помощью параметра (выбирать количество ячеек HyperLogLog).
|
||||
* Добавлена таблица `system.contributors`, содержащая имена всех, кто делал коммиты в ClickHouse.
|
||||
* Добавлена возможность не указывать партицию для запроса `ALTER TABLE ... FREEZE` для бэкапа сразу всех партиций.
|
||||
* Добавлены функции `dictGet`, `dictGetOrDefault` без указания типа возвращаемого значения. Тип определяется автоматически из описания словаря.
|
||||
* Возможность указания комментария для столбца в описании таблицы и изменения его с помощью `ALTER`.
|
||||
* Возможность чтения из таблицы типа `Join` в случае простых ключей.
|
||||
* Возможность указания настроек `join_use_nulls`, `max_rows_in_join`, `max_bytes_in_join`, `join_overflow_mode` при создании таблицы типа `Join`.
|
||||
* Добавлена функция `joinGet`, позволяющая использовать таблицы типа `Join` как словарь.
|
||||
* Добавлены столбцы `partition_key`, `sorting_key`, `primary_key`, `sampling_key` в таблицу `system.tables`, позволяющие получить информацию о ключах таблицы.
|
||||
* Добавлены столбцы `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, `is_in_sampling_key` в таблицу `system.columns`.
|
||||
* Добавлены столбцы `min_time`, `max_time` в таблицу `system.parts`. Эти столбцы заполняются, если ключ партиционирования является выражением от столбцов типа `DateTime`.
|
||||
|
||||
### Исправления ошибок:
|
||||
|
||||
* Исправления и улучшения производительности для типа данных `LowCardinality`. `GROUP BY` по `LowCardinality(Nullable(...))`. Получение `extremes` значений. Выполнение функций высшего порядка. `LEFT ARRAY JOIN`. Распределённый `GROUP BY`. Функции, возвращающие `Array`. Выполнение `ORDER BY`. Запись в `Distributed` таблицы (nicelulu). Обратная совместимость для запросов `INSERT` от старых клиентов, реализующих `Native` протокол. Поддержка `LowCardinality` для `JOIN`. Производительность при работе в один поток.
|
||||
* Исправлена работа настройки `select_sequential_consistency`. Ранее, при включенной настройке, после начала записи в новую партицию, мог возвращаться неполный результат.
|
||||
* Корректное указание базы данных при выполнении DDL запросов `ON CLUSTER`, а также при выполнении `ALTER UPDATE/DELETE`.
|
||||
* Корректное указание базы данных для подзапросов внутри VIEW.
|
||||
* Исправлена работа `PREWHERE` с `FINAL` для `VersionedCollapsingMergeTree`.
|
||||
* Возможность с помощью запроса `KILL QUERY` отмены запросов, которые ещё не начали выполняться из-за ожидания блокировки таблицы.
|
||||
* Исправлены расчёты с датой и временем в случае, если стрелки часов были переведены назад в полночь (это происходит в Иране, а также было Москве с 1981 по 1983 год). Ранее это приводило к тому, что стрелки часов переводились на сутки раньше, чем нужно, а также приводило к некорректному форматированию даты-с-временем в текстовом виде.
|
||||
* Исправлена работа некоторых случаев `VIEW` и подзапросов без указания базы данных.
|
||||
* Исправлен race condition при одновременном чтении из `MATERIALIZED VIEW` и удалением `MATERIALIZED VIEW` из-за отсутствия блокировки внутренней таблицы `MATERIALIZED VIEW`.
|
||||
* Исправлена ошибка `Lock handler cannot be nullptr.`
|
||||
* Исправления выполнения запросов при включенной настройке `compile_expressions` (выключена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`.
|
||||
* Исправлено падение при указании неконстантного аргумента scale в функциях `toDecimal32/64/128`.
|
||||
* Исправлена ошибка при попытке вставки в формате `Values` массива с `NULL` элементами в столбец типа `Array` без `Nullable` (в случае `input_format_values_interpret_expressions` = 1).
|
||||
* Исправлено непрерывное логгирование ошибок в `DDLWorker`, если ZooKeeper недоступен.
|
||||
* Исправлен тип возвращаемого значения для функций `quantile*` от аргументов типа `Date` и `DateTime`.
|
||||
* Исправлена работа секции `WITH`, если она задаёт простой алиас без выражений.
|
||||
* Исправлена обработка запросов с именованными подзапросами и квалифицированными именами столбцов при включенной настройке `enable_optimize_predicate_expression`.
|
||||
* Исправлена ошибка `Attempt to attach to nullptr thread group` при работе материализованных представлений.
|
||||
* Исправлено падение при передаче некоторых некорректных аргументов в функцию `arrayReverse`.
|
||||
* Исправлен buffer overflow в функции `extractURLParameter`. Увеличена производительность. Добавлена корректная обработка строк, содержащих нулевые байты.
|
||||
* Исправлен buffer overflow в функциях `lowerUTF8`, `upperUTF8`. Удалена возможность выполнения этих функций над аргументами типа `FixedString`.
|
||||
* Исправлен редкий race condition при удалении таблиц типа `MergeTree`.
|
||||
* Исправлен race condition при чтении из таблиц типа `Buffer` и одновременном `ALTER` либо `DROP` таблиц назначения.
|
||||
* Исправлен segfault в случае превышения ограничения `max_temporary_non_const_columns`.
|
||||
|
||||
### Улучшения:
|
||||
|
||||
* Обработанные конфигурационные файлы записываются сервером не в `/etc/clickhouse-server/` директорию, а в директорию `preprocessed_configs` внутри `path`. Это позволяет оставить директорию `/etc/clickhouse-server/` недоступной для записи пользователем `clickhouse`, что повышает безопасность.
|
||||
* Настройка `min_merge_bytes_to_use_direct_io` выставлена по-умолчанию в 10 GiB. Слияния, образующие крупные куски таблиц семейства MergeTree, будут производиться в режиме `O_DIRECT`, что исключает вымывание кэша.
|
||||
* Ускорен запуск сервера в случае наличия очень большого количества таблиц.
|
||||
* Добавлен пул соединений и HTTP `Keep-Alive` для соединения между репликами.
|
||||
* В случае ошибки синтаксиса запроса, в `HTTP` интерфейсе возвращается код `400 Bad Request` (ранее возвращался код 500).
|
||||
* Для настройки `join_default_strictness` выбрано значение по-умолчанию `ALL` для совместимости.
|
||||
* Убрано логгирование в `stderr` из библиотеки `re2` в случае некорректных или сложных регулярных выражений.
|
||||
* Для движка таблиц `Kafka` TODO
|
||||
* Функции `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64` теперь работают для произвольного количества аргументов, а также для аргументов-кортежей.
|
||||
* Функция `arrayReverse` теперь работает с любыми типами массивов.
|
||||
* Добавлен опциональный параметр - размер слота для функции `timeSlots`.
|
||||
* Для `FULL` и `RIGHT JOIN` учитывается настройка `max_block_size` для потока неприсоединённых данных из правой таблицы.
|
||||
* В `clickhouse-benchmark` и `clickhouse-performance-test` добавлен параметр командной строки `--secure` для включения TLS.
|
||||
* Преобразование типов в случае, если структура таблицы типа `Buffer` не соответствует структуре таблицы назначения.
|
||||
* Добавлена настройка `tcp_keep_alive_timeout` для включения keep-alive пакетов после неактивности в течение указанного интервала времени.
|
||||
* Убрано излишнее квотирование значений ключа партиции в таблице `system.parts`, если он состоит из одного столбца.
|
||||
* Функция деления с остатком работает для типов данных `Date` и `DateTime`.
|
||||
* Добавлены синонимы функций `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, `MID`. Некоторые имена функций сделаны регистронезависимыми для совместимости со стандартом SQL. Добавлен синтаксический сахар `SUBSTRING(expr FROM start FOR length)` для совместимости с SQL.
|
||||
* Добавлена возможность фиксации (`mlock`) страниц памяти, соответствующих исполняемому коду `clickhouse-server` для предотвращения вытеснения их из памяти. Возможность выключена по-умолчанию.
|
||||
* Увеличена производительность чтения с `O_DIRECT` (с включенной опцией `min_bytes_to_use_direct_io`).
|
||||
* Улучшена производительность работы функции `dictGet...OrDefault` в случае константного аргумента-ключа и неконстантного аргумента-default.
|
||||
* В функции `firstSignificantSubdomain` добавлена обработка доменов `gov`, `mil`, `edu`. Увеличена производительность работы.
|
||||
* Возможность указания произвольных переменных окружения для запуска `clickhouse-server` посредством `SYS-V init.d`-скрипта с помощью указания `CLICKHOUSE_PROGRAM_ENV` в `/etc/default/clickhouse`.
|
||||
* Правильный код возврата init-скрипта clickhouse-server.
|
||||
* В таблицу `system.metrics` добавлена метрика `VersionInteger`, а в `system.build_options` добавлена строчка `VERSION_INTEGER`, содержащая версию ClickHouse в числовом представлении, вида `18016000`.
|
||||
* Удалена возможность сравнения типа `Date` с числом, чтобы избежать потенциальных ошибок вида `date = 2018-12-17`, где ошибочно не указаны кавычки вокруг даты.
|
||||
* Исправлено поведение функций с состоянием типа `rowNumberInAllBlocks` - раньше они выдавали число на единицу больше вследствие их запуска во время анализа запроса.
|
||||
* При невозможности удалить файл `force_restore_data`, выводится сообщение об ошибке.
|
||||
|
||||
### Улучшение сборки:
|
||||
|
||||
* Обновлена библиотека `jemalloc`, что исправляет потенциальную утечку памяти.
|
||||
* Для debug сборок включено по-умолчанию профилирование `jemalloc`.
|
||||
* Добавлена возможность запуска интеграционных тестов, при наличии установленным в системе лишь `Docker`.
|
||||
* Добавлен fuzz тест выражений в SELECT запросах.
|
||||
* Добавлен покоммитный стресс-тест, выполняющий функциональные тесты параллельно и в произвольном порядке, позволяющий обнаружить больше race conditions.
|
||||
* Улучшение способа запуска clickhouse-server в Docker образе.
|
||||
* Для Docker образа добавлена поддержка инициализации базы данных с помощью файлов в директории `/docker-entrypoint-initdb.d`.
|
||||
* Исправления для сборки под ARM.
|
||||
|
||||
|
||||
## ClickHouse release 18.14.18, 2018-12-04
|
||||
|
||||
### Исправления ошибок:
|
||||
@ -897,7 +988,7 @@
|
||||
|
||||
### Новые возможности:
|
||||
* Произвольный ключ партиционирования для таблиц семейства MergeTree.
|
||||
* Движок таблиц [Kafka](https://clickhouse.yandex/docs/en/single/index.html#document-table_engines/kafka).
|
||||
* Движок таблиц [Kafka](https://clickhouse.yandex/docs/en/operations/table_engines/kafka/).
|
||||
* Возможность загружать модели [CatBoost](https://catboost.yandex/) и применять их к данным, хранящимся в ClickHouse.
|
||||
* Поддержка часовых поясов с нецелым смещением от UTC.
|
||||
* Поддержка операций с временными интервалами.
|
||||
|
@ -127,7 +127,10 @@ endif ()
|
||||
|
||||
include (cmake/test_cpu.cmake)
|
||||
|
||||
option (ARCH_NATIVE "Enable -march=native compiler flag" ${ARCH_ARM})
|
||||
if(NOT COMPILER_CLANG) # clang: error: the clang compiler does not support '-march=native'
|
||||
option(ARCH_NATIVE "Enable -march=native compiler flag" ${ARCH_ARM})
|
||||
endif()
|
||||
|
||||
if (ARCH_NATIVE)
|
||||
set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native")
|
||||
endif ()
|
||||
@ -187,10 +190,10 @@ if (OS_LINUX AND COMPILER_CLANG)
|
||||
endif ()
|
||||
endif ()
|
||||
|
||||
if (COMPILER_GCC)
|
||||
set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++")
|
||||
else ()
|
||||
if (USE_LIBCXX)
|
||||
set (STATIC_STDLIB_FLAGS "")
|
||||
else ()
|
||||
set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++")
|
||||
endif ()
|
||||
|
||||
if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD))
|
||||
|
@ -25,8 +25,8 @@ if (SANITIZE)
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libtsan")
|
||||
endif ()
|
||||
elseif (SANITIZE STREQUAL "undefined")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=undefined")
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=undefined")
|
||||
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=undefined -fno-sanitize-recover=all")
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=undefined -fno-sanitize-recover=all")
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=undefined")
|
||||
if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libubsan")
|
||||
|
@ -291,6 +291,11 @@ target_include_directories (clickhouse_common_io BEFORE PRIVATE ${COMMON_INCLUDE
|
||||
add_subdirectory (programs)
|
||||
add_subdirectory (tests)
|
||||
|
||||
if (GLIBC_COMPATIBILITY AND NOT CLICKHOUSE_SPLIT_BINARY)
|
||||
MESSAGE(STATUS "Some symbols from glibc will be replaced for compatibility")
|
||||
target_link_libraries(dbms PUBLIC glibc-compatibility)
|
||||
endif()
|
||||
|
||||
if (ENABLE_TESTS)
|
||||
macro (grep_gtest_sources BASE_DIR DST_VAR)
|
||||
# Cold match files that are not in tests/ directories
|
||||
|
@ -1,11 +1,11 @@
|
||||
# This strings autochanged from release_lib.sh:
|
||||
set(VERSION_REVISION 54410 CACHE STRING "") # changed manually for tests
|
||||
set(VERSION_REVISION 54412 CACHE STRING "") # changed manually for tests
|
||||
set(VERSION_MAJOR 18 CACHE STRING "")
|
||||
set(VERSION_MINOR 14 CACHE STRING "")
|
||||
set(VERSION_PATCH 17 CACHE STRING "")
|
||||
set(VERSION_GITHASH ac2895d769c3dcf070530dec7fcfdcf87bfa852a CACHE STRING "")
|
||||
set(VERSION_DESCRIBE v18.14.17-testing CACHE STRING "")
|
||||
set(VERSION_STRING 18.14.17 CACHE STRING "")
|
||||
set(VERSION_MINOR 16 CACHE STRING "")
|
||||
set(VERSION_PATCH 0 CACHE STRING "")
|
||||
set(VERSION_GITHASH b9b48c646c253358340bd39fd57754e92f88cd8a CACHE STRING "")
|
||||
set(VERSION_DESCRIBE v18.16.0-testing CACHE STRING "")
|
||||
set(VERSION_STRING 18.16.0 CACHE STRING "")
|
||||
# end of autochange
|
||||
|
||||
set(VERSION_EXTRA "" CACHE STRING "")
|
||||
|
@ -617,8 +617,14 @@ private:
|
||||
{
|
||||
std::cerr << std::endl
|
||||
<< "Exception on client:" << std::endl
|
||||
<< "Code: " << e.code() << ". " << e.displayText() << std::endl
|
||||
<< std::endl;
|
||||
<< "Code: " << e.code() << ". " << e.displayText() << std::endl;
|
||||
|
||||
if (config().getBool("stacktrace", false))
|
||||
std::cerr << "Stack trace:" << std::endl
|
||||
<< e.getStackTrace().toString() << std::endl;
|
||||
|
||||
std::cerr << std::endl;
|
||||
|
||||
}
|
||||
|
||||
/// Client-side exception during query execution can result in the loss of
|
||||
|
@ -481,7 +481,7 @@ String DB::TaskShard::getHostNameExample() const
|
||||
}
|
||||
|
||||
|
||||
static bool isExtedndedDefinitionStorage(const ASTPtr & storage_ast)
|
||||
static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast)
|
||||
{
|
||||
const ASTStorage & storage = typeid_cast<const ASTStorage &>(*storage_ast);
|
||||
return storage.partition_by || storage.order_by || storage.sample_by;
|
||||
@ -503,7 +503,7 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast)
|
||||
ASTPtr arguments_ast = engine.arguments->clone();
|
||||
ASTs & arguments = typeid_cast<ASTExpressionList &>(*arguments_ast).children;
|
||||
|
||||
if (isExtedndedDefinitionStorage(storage_ast))
|
||||
if (isExtendedDefinitionStorage(storage_ast))
|
||||
{
|
||||
if (storage.partition_by)
|
||||
return storage.partition_by->clone();
|
||||
|
@ -121,9 +121,6 @@ void ODBCBridge::initialize(Application & self)
|
||||
if (is_help)
|
||||
return;
|
||||
|
||||
if (!config().has("logger.log"))
|
||||
config().setBool("logger.console", true);
|
||||
|
||||
config().setString("logger", "ODBCBridge");
|
||||
|
||||
buildLoggers(config());
|
||||
|
@ -36,7 +36,7 @@ template <typename T>
|
||||
struct GroupArrayNumericData
|
||||
{
|
||||
// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena
|
||||
using Allocator = MixedArenaAllocator<4096>;
|
||||
using Allocator = MixedAlignedArenaAllocator<alignof(T), 4096>;
|
||||
using Array = PODArray<T, 32, Allocator>;
|
||||
|
||||
Array value;
|
||||
|
@ -38,7 +38,7 @@ struct MaxIntersectionsData
|
||||
using Value = std::pair<T, Int64>;
|
||||
|
||||
// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena
|
||||
using Allocator = MixedArenaAllocator<4096>;
|
||||
using Allocator = MixedAlignedArenaAllocator<alignof(Value), 4096>;
|
||||
using Array = PODArray<Value, 32, Allocator>;
|
||||
|
||||
Array value;
|
||||
|
@ -0,0 +1,3 @@
|
||||
if (ENABLE_TESTS)
|
||||
add_subdirectory (tests)
|
||||
endif ()
|
@ -8,6 +8,8 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
|
||||
#include <common/unaligned.h>
|
||||
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
@ -186,7 +188,7 @@ StringRef ColumnArray::serializeValueIntoArena(size_t n, Arena & arena, char con
|
||||
|
||||
const char * ColumnArray::deserializeAndInsertFromArena(const char * pos)
|
||||
{
|
||||
size_t array_size = *reinterpret_cast<const size_t *>(pos);
|
||||
size_t array_size = unalignedLoad<size_t>(pos);
|
||||
pos += sizeof(array_size);
|
||||
|
||||
for (size_t i = 0; i < array_size; ++i)
|
||||
|
@ -2,12 +2,15 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <common/unaligned.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
|
||||
template <typename T> bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale);
|
||||
|
||||
namespace DB
|
||||
@ -41,7 +44,7 @@ StringRef ColumnDecimal<T>::serializeValueIntoArena(size_t n, Arena & arena, cha
|
||||
template <typename T>
|
||||
const char * ColumnDecimal<T>::deserializeAndInsertFromArena(const char * pos)
|
||||
{
|
||||
data.push_back(*reinterpret_cast<const T *>(pos));
|
||||
data.push_back(unalignedLoad<T>(pos));
|
||||
return pos + sizeof(T);
|
||||
}
|
||||
|
||||
|
@ -212,13 +212,6 @@ void ColumnLowCardinality::insertData(const char * pos, size_t length)
|
||||
idx.check(getDictionary().size());
|
||||
}
|
||||
|
||||
void ColumnLowCardinality::insertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
{
|
||||
compactIfSharedDictionary();
|
||||
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertDataWithTerminatingZero(pos, length));
|
||||
idx.check(getDictionary().size());
|
||||
}
|
||||
|
||||
StringRef ColumnLowCardinality::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
||||
{
|
||||
return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin);
|
||||
|
@ -73,8 +73,6 @@ public:
|
||||
void insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions);
|
||||
|
||||
void insertData(const char * pos, size_t length) override;
|
||||
void insertDataWithTerminatingZero(const char * pos, size_t length) override;
|
||||
|
||||
|
||||
void popBack(size_t n) override { idx.popBack(n); }
|
||||
|
||||
|
@ -5,6 +5,8 @@
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
#include <common/unaligned.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -176,7 +178,7 @@ StringRef ColumnString::serializeValueIntoArena(size_t n, Arena & arena, char co
|
||||
|
||||
const char * ColumnString::deserializeAndInsertFromArena(const char * pos)
|
||||
{
|
||||
const size_t string_size = *reinterpret_cast<const size_t *>(pos);
|
||||
const size_t string_size = unalignedLoad<size_t>(pos);
|
||||
pos += sizeof(string_size);
|
||||
|
||||
const size_t old_size = chars.size();
|
||||
|
@ -158,7 +158,8 @@ public:
|
||||
offsets.push_back(new_size);
|
||||
}
|
||||
|
||||
void insertDataWithTerminatingZero(const char * pos, size_t length) override
|
||||
/// Like getData, but inserting data should be zero-ending (i.e. length is 1 byte greater than real string size).
|
||||
void insertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
{
|
||||
const size_t old_size = chars.size();
|
||||
const size_t new_size = old_size + length;
|
||||
|
@ -13,6 +13,9 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
#include <common/unaligned.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -44,7 +47,6 @@ public:
|
||||
IColumnUnique::IndexesWithOverflow uniqueInsertRangeWithOverflow(const IColumn & src, size_t start, size_t length,
|
||||
size_t max_dictionary_size) override;
|
||||
size_t uniqueInsertData(const char * pos, size_t length) override;
|
||||
size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) override;
|
||||
size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) override;
|
||||
|
||||
size_t getDefaultValueIndex() const override { return 0; }
|
||||
@ -100,6 +102,7 @@ private:
|
||||
|
||||
ColumnPtr column_holder;
|
||||
bool is_nullable;
|
||||
size_t size_of_value_if_fixed = 0;
|
||||
ReverseIndex<UInt64, ColumnType> index;
|
||||
|
||||
/// For DataTypeNullable, stores null map.
|
||||
@ -151,6 +154,7 @@ template <typename ColumnType>
|
||||
ColumnUnique<ColumnType>::ColumnUnique(const ColumnUnique & other)
|
||||
: column_holder(other.column_holder)
|
||||
, is_nullable(other.is_nullable)
|
||||
, size_of_value_if_fixed (other.size_of_value_if_fixed)
|
||||
, index(numSpecialValues(is_nullable), 0)
|
||||
{
|
||||
index.setColumn(getRawColumnPtr());
|
||||
@ -166,6 +170,9 @@ ColumnUnique<ColumnType>::ColumnUnique(const IDataType & type)
|
||||
column_holder = holder_type.createColumn()->cloneResized(numSpecialValues());
|
||||
index.setColumn(getRawColumnPtr());
|
||||
createNullMask();
|
||||
|
||||
if (column_holder->valuesHaveFixedSize())
|
||||
size_of_value_if_fixed = column_holder->sizeOfValueIfFixed();
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
@ -181,6 +188,9 @@ ColumnUnique<ColumnType>::ColumnUnique(MutableColumnPtr && holder, bool is_nulla
|
||||
|
||||
index.setColumn(getRawColumnPtr());
|
||||
createNullMask();
|
||||
|
||||
if (column_holder->valuesHaveFixedSize())
|
||||
size_of_value_if_fixed = column_holder->sizeOfValueIfFixed();
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
@ -243,20 +253,11 @@ size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
||||
if (x.getType() == Field::Types::Null)
|
||||
return getNullValueIndex();
|
||||
|
||||
auto column = getRawColumnPtr();
|
||||
auto prev_size = static_cast<UInt64>(column->size());
|
||||
if (size_of_value_if_fixed)
|
||||
return uniqueInsertData(&x.get<char>(), size_of_value_if_fixed);
|
||||
|
||||
if ((*column)[getNestedTypeDefaultValueIndex()] == x)
|
||||
return getNestedTypeDefaultValueIndex();
|
||||
|
||||
column->insert(x);
|
||||
auto pos = index.insert(prev_size);
|
||||
if (pos != prev_size)
|
||||
column->popBack(1);
|
||||
|
||||
updateNullMask();
|
||||
|
||||
return pos;
|
||||
auto & val = x.get<String>();
|
||||
return uniqueInsertData(val.data(), val.size());
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
@ -280,50 +281,13 @@ size_t ColumnUnique<ColumnType>::uniqueInsertData(const char * pos, size_t lengt
|
||||
if (column->getDataAt(getNestedTypeDefaultValueIndex()) == StringRef(pos, length))
|
||||
return getNestedTypeDefaultValueIndex();
|
||||
|
||||
UInt64 size = column->size();
|
||||
UInt64 insertion_point = index.getInsertionPoint(StringRef(pos, length));
|
||||
|
||||
if (insertion_point == size)
|
||||
{
|
||||
column->insertData(pos, length);
|
||||
index.insertFromLastRow();
|
||||
}
|
||||
auto insertion_point = index.insert(StringRef(pos, length));
|
||||
|
||||
updateNullMask();
|
||||
|
||||
return insertion_point;
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
{
|
||||
if (std::is_same<ColumnType, ColumnString>::value)
|
||||
return uniqueInsertData(pos, length - 1);
|
||||
|
||||
if (column_holder->valuesHaveFixedSize())
|
||||
return uniqueInsertData(pos, length);
|
||||
|
||||
/// Don't know if data actually has terminating zero. So, insert it firstly.
|
||||
|
||||
auto column = getRawColumnPtr();
|
||||
size_t prev_size = column->size();
|
||||
column->insertDataWithTerminatingZero(pos, length);
|
||||
|
||||
if (column->compareAt(getNestedTypeDefaultValueIndex(), prev_size, *column, 1) == 0)
|
||||
{
|
||||
column->popBack(1);
|
||||
return getNestedTypeDefaultValueIndex();
|
||||
}
|
||||
|
||||
auto position = index.insert(prev_size);
|
||||
if (position != prev_size)
|
||||
column->popBack(1);
|
||||
|
||||
updateNullMask();
|
||||
|
||||
return static_cast<size_t>(position);
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
StringRef ColumnUnique<ColumnType>::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
||||
{
|
||||
@ -362,23 +326,20 @@ size_t ColumnUnique<ColumnType>::uniqueDeserializeAndInsertFromArena(const char
|
||||
}
|
||||
}
|
||||
|
||||
auto column = getRawColumnPtr();
|
||||
size_t prev_size = column->size();
|
||||
new_pos = column->deserializeAndInsertFromArena(pos);
|
||||
|
||||
if (column->compareAt(getNestedTypeDefaultValueIndex(), prev_size, *column, 1) == 0)
|
||||
/// Numbers, FixedString
|
||||
if (size_of_value_if_fixed)
|
||||
{
|
||||
column->popBack(1);
|
||||
return getNestedTypeDefaultValueIndex();
|
||||
new_pos = pos + size_of_value_if_fixed;
|
||||
return uniqueInsertData(pos, size_of_value_if_fixed);
|
||||
}
|
||||
|
||||
auto index_pos = index.insert(prev_size);
|
||||
if (index_pos != prev_size)
|
||||
column->popBack(1);
|
||||
/// String
|
||||
const size_t string_size = unalignedLoad<size_t>(pos);
|
||||
pos += sizeof(string_size);
|
||||
new_pos = pos + string_size;
|
||||
|
||||
updateNullMask();
|
||||
|
||||
return static_cast<size_t>(index_pos);
|
||||
/// -1 because of terminating zero
|
||||
return uniqueInsertData(pos, string_size - 1);
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
@ -482,20 +443,14 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||
if (secondary_index)
|
||||
next_position += secondary_index->size();
|
||||
|
||||
auto check_inserted_position = [&next_position](UInt64 inserted_position)
|
||||
auto insert_key = [&](const StringRef & ref, ReverseIndex<UInt64, ColumnType> & cur_index) -> MutableColumnPtr
|
||||
{
|
||||
if (inserted_position != next_position)
|
||||
throw Exception("Inserted position " + toString(inserted_position)
|
||||
+ " is not equal with expected " + toString(next_position), ErrorCodes::LOGICAL_ERROR);
|
||||
};
|
||||
auto inserted_pos = cur_index.insert(ref);
|
||||
positions[num_added_rows] = inserted_pos;
|
||||
if (inserted_pos == next_position)
|
||||
return update_position(next_position);
|
||||
|
||||
auto insert_key = [&](const StringRef & ref, ReverseIndex<UInt64, ColumnType> * cur_index)
|
||||
{
|
||||
positions[num_added_rows] = next_position;
|
||||
cur_index->getColumn()->insertData(ref.data, ref.size);
|
||||
auto inserted_pos = cur_index->insertFromLastRow();
|
||||
check_inserted_position(inserted_pos);
|
||||
return update_position(next_position);
|
||||
return nullptr;
|
||||
};
|
||||
|
||||
for (; num_added_rows < length; ++num_added_rows)
|
||||
@ -509,29 +464,21 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||
else
|
||||
{
|
||||
auto ref = src_column->getDataAt(row);
|
||||
auto cur_index = &index;
|
||||
bool inserted = false;
|
||||
MutableColumnPtr res = nullptr;
|
||||
|
||||
while (!inserted)
|
||||
if (secondary_index && next_position >= max_dictionary_size)
|
||||
{
|
||||
auto insertion_point = cur_index->getInsertionPoint(ref);
|
||||
|
||||
if (insertion_point == cur_index->lastInsertionPoint())
|
||||
{
|
||||
if (secondary_index && cur_index != secondary_index && next_position >= max_dictionary_size)
|
||||
{
|
||||
cur_index = secondary_index;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (auto res = insert_key(ref, cur_index))
|
||||
return res;
|
||||
}
|
||||
auto insertion_point = index.getInsertionPoint(ref);
|
||||
if (insertion_point == index.lastInsertionPoint())
|
||||
res = insert_key(ref, *secondary_index);
|
||||
else
|
||||
positions[num_added_rows] = insertion_point;
|
||||
|
||||
inserted = true;
|
||||
positions[num_added_rows] = insertion_point;
|
||||
}
|
||||
else
|
||||
res = insert_key(ref, index);
|
||||
|
||||
if (res)
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,8 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <cmath>
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
#include <common/unaligned.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -164,7 +164,7 @@ public:
|
||||
|
||||
void insertData(const char * pos, size_t /*length*/) override
|
||||
{
|
||||
data.push_back(*reinterpret_cast<const T *>(pos));
|
||||
data.push_back(unalignedLoad<T>(pos));
|
||||
}
|
||||
|
||||
void insertDefault() override
|
||||
|
@ -143,13 +143,6 @@ public:
|
||||
/// Parameter length could be ignored if column values have fixed size.
|
||||
virtual void insertData(const char * pos, size_t length) = 0;
|
||||
|
||||
/// Like getData, but has special behavior for columns that contain variable-length strings.
|
||||
/// In this special case inserting data should be zero-ending (i.e. length is 1 byte greater than real string size).
|
||||
virtual void insertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
{
|
||||
insertData(pos, length);
|
||||
}
|
||||
|
||||
/// Appends "default value".
|
||||
/// Is used when there are need to increase column size, but inserting value doesn't make sense.
|
||||
/// For example, ColumnNullable(Nested) absolutely ignores values of nested column if it is marked as NULL.
|
||||
|
@ -51,7 +51,6 @@ public:
|
||||
/// Is used to optimize some computations (in aggregation, for example).
|
||||
/// Parameter length could be ignored if column values have fixed size.
|
||||
virtual size_t uniqueInsertData(const char * pos, size_t length) = 0;
|
||||
virtual size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) = 0;
|
||||
|
||||
virtual size_t getDefaultValueIndex() const = 0; /// Nullable ? getNullValueIndex : getNestedTypeDefaultValueIndex
|
||||
virtual size_t getNullValueIndex() const = 0; /// Throws if not nullable.
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <ext/range.h>
|
||||
#include <common/unaligned.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -56,32 +58,15 @@ namespace
|
||||
};
|
||||
|
||||
|
||||
template <typename Hash>
|
||||
struct ReverseIndexHash : public Hash
|
||||
struct ReverseIndexHash
|
||||
{
|
||||
template <typename T>
|
||||
size_t operator()(T) const
|
||||
{
|
||||
throw Exception("operator()(key) is not implemented for ReverseIndexHash.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
template <typename State, typename T>
|
||||
size_t operator()(const State & state, T key) const
|
||||
{
|
||||
auto index = key;
|
||||
if constexpr (State::has_base_index)
|
||||
index -= state.base_index;
|
||||
|
||||
return Hash::operator()(state.index_column->getElement(index));
|
||||
}
|
||||
};
|
||||
|
||||
using ReverseIndexStringHash = ReverseIndexHash<StringRefHash>;
|
||||
|
||||
template <typename IndexType>
|
||||
using ReverseIndexNumberHash = ReverseIndexHash<DefaultHash<IndexType>>;
|
||||
|
||||
|
||||
template <typename IndexType, typename Hash, typename HashTable, typename ColumnType, bool string_hash, bool has_base_index>
|
||||
struct ReverseIndexHashTableCell
|
||||
: public HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>
|
||||
@ -99,6 +84,7 @@ namespace
|
||||
static_assert(!std::is_same_v<typename std::decay<T>::type, typename std::decay<IndexType>::type>);
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Special case when we want to compare with something not in index_column.
|
||||
/// When we compare something inside column default keyEquals checks only that row numbers are equal.
|
||||
bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const
|
||||
@ -126,7 +112,11 @@ namespace
|
||||
if constexpr (string_hash)
|
||||
return (*state.saved_hash_column)[index];
|
||||
else
|
||||
return hash(state, key);
|
||||
{
|
||||
using ValueType = typename ColumnType::value_type;
|
||||
ValueType value = unalignedLoad<ValueType>(state.index_column->getDataAt(index).data);
|
||||
return DefaultHash<ValueType>()(value);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
@ -147,28 +137,28 @@ namespace
|
||||
IndexType,
|
||||
ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexStringHash,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
true,
|
||||
has_base_index>,
|
||||
ReverseIndexStringHash>
|
||||
ReverseIndexHash>
|
||||
{
|
||||
using Base = HashTableWithPublicState<
|
||||
IndexType,
|
||||
ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexStringHash,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
true,
|
||||
has_base_index>,
|
||||
ReverseIndexStringHash>;
|
||||
ReverseIndexHash>;
|
||||
public:
|
||||
using Base::Base;
|
||||
friend struct ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexStringHash,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
true,
|
||||
@ -180,28 +170,28 @@ namespace
|
||||
IndexType,
|
||||
ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
false,
|
||||
has_base_index>,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>>
|
||||
ReverseIndexHash>
|
||||
{
|
||||
using Base = HashTableWithPublicState<
|
||||
IndexType,
|
||||
ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
false,
|
||||
has_base_index>,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>>;
|
||||
ReverseIndexHash>;
|
||||
public:
|
||||
using Base::Base;
|
||||
friend struct ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
false,
|
||||
@ -253,8 +243,7 @@ public:
|
||||
static constexpr bool is_numeric_column = isNumericColumn(static_cast<ColumnType *>(nullptr));
|
||||
static constexpr bool use_saved_hash = !is_numeric_column;
|
||||
|
||||
UInt64 insert(UInt64 from_position); /// Insert into index column[from_position];
|
||||
UInt64 insertFromLastRow();
|
||||
UInt64 insert(const StringRef & data);
|
||||
UInt64 getInsertionPoint(const StringRef & data);
|
||||
UInt64 lastInsertionPoint() const { return size() + base_index; }
|
||||
|
||||
@ -302,7 +291,7 @@ private:
|
||||
if constexpr (is_numeric_column)
|
||||
{
|
||||
using ValueType = typename ColumnType::value_type;
|
||||
ValueType value = *reinterpret_cast<const ValueType *>(ref.data);
|
||||
ValueType value = unalignedLoad<ValueType>(ref.data);
|
||||
return DefaultHash<ValueType>()(value);
|
||||
}
|
||||
else
|
||||
@ -367,7 +356,7 @@ void ReverseIndex<IndexType, ColumnType>::buildIndex()
|
||||
else
|
||||
hash = getHash(column->getDataAt(row));
|
||||
|
||||
index->emplace(row + base_index, iterator, inserted, hash);
|
||||
index->emplace(row + base_index, iterator, inserted, hash, column->getDataAt(row));
|
||||
|
||||
if (!inserted)
|
||||
throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -390,7 +379,7 @@ ColumnUInt64::MutablePtr ReverseIndex<IndexType, ColumnType>::calcHashes() const
|
||||
}
|
||||
|
||||
template <typename IndexType, typename ColumnType>
|
||||
UInt64 ReverseIndex<IndexType, ColumnType>::insert(UInt64 from_position)
|
||||
UInt64 ReverseIndex<IndexType, ColumnType>::insert(const StringRef & data)
|
||||
{
|
||||
if (!index)
|
||||
buildIndex();
|
||||
@ -399,42 +388,35 @@ UInt64 ReverseIndex<IndexType, ColumnType>::insert(UInt64 from_position)
|
||||
IteratorType iterator;
|
||||
bool inserted;
|
||||
|
||||
auto hash = getHash(column->getDataAt(from_position));
|
||||
auto hash = getHash(data);
|
||||
UInt64 num_rows = size();
|
||||
|
||||
if constexpr (use_saved_hash)
|
||||
{
|
||||
auto & data = saved_hash->getData();
|
||||
if (data.size() <= from_position)
|
||||
data.resize(from_position + 1);
|
||||
data[from_position] = hash;
|
||||
if (data.size() <= num_rows)
|
||||
data.resize(num_rows + 1);
|
||||
data[num_rows] = hash;
|
||||
}
|
||||
else
|
||||
column->insertData(data.data, data.size);
|
||||
|
||||
index->emplace(num_rows + base_index, iterator, inserted, hash, data);
|
||||
|
||||
if constexpr (use_saved_hash)
|
||||
{
|
||||
if (inserted)
|
||||
column->insertData(data.data, data.size);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!inserted)
|
||||
column->popBack(1);
|
||||
}
|
||||
|
||||
index->emplace(from_position + base_index, iterator, inserted, hash);
|
||||
|
||||
return *iterator;
|
||||
}
|
||||
|
||||
template <typename IndexType, typename ColumnType>
|
||||
UInt64 ReverseIndex<IndexType, ColumnType>::insertFromLastRow()
|
||||
{
|
||||
if (!column)
|
||||
throw Exception("ReverseIndex can't insert row from column because index column wasn't set.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
UInt64 num_rows = size();
|
||||
|
||||
if (num_rows == 0)
|
||||
throw Exception("ReverseIndex can't insert row from column because it is empty.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
UInt64 position = num_rows - 1;
|
||||
UInt64 inserted_pos = insert(position);
|
||||
if (position + base_index != inserted_pos)
|
||||
throw Exception("Can't insert into reverse index from last row (" + toString(position + base_index)
|
||||
+ ") because the same row is in position " + toString(inserted_pos), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return inserted_pos;
|
||||
}
|
||||
|
||||
template <typename IndexType, typename ColumnType>
|
||||
UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPoint(const StringRef & data)
|
||||
{
|
||||
|
4
dbms/src/Columns/tests/CMakeLists.txt
Normal file
4
dbms/src/Columns/tests/CMakeLists.txt
Normal file
@ -0,0 +1,4 @@
|
||||
set(SRCS)
|
||||
|
||||
add_executable (column_unique column_unique.cpp ${SRCS})
|
||||
target_link_libraries (column_unique PRIVATE dbms gtest_main)
|
193
dbms/src/Columns/tests/column_unique.cpp
Normal file
193
dbms/src/Columns/tests/column_unique.cpp
Normal file
@ -0,0 +1,193 @@
|
||||
#include <Columns/ColumnUnique.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
|
||||
#endif
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
using namespace DB;
|
||||
|
||||
TEST(column_unique, column_unique_unique_insert_range_Test)
|
||||
{
|
||||
std::unordered_map<String, size_t> ref_map;
|
||||
auto data_type = std::make_shared<DataTypeString>();
|
||||
auto column_unique = ColumnUnique<ColumnString>::create(*data_type);
|
||||
auto column_string = ColumnString::create();
|
||||
|
||||
size_t num_values = 1000000;
|
||||
size_t mod_to = 1000;
|
||||
|
||||
std::vector<size_t> indexes(num_values);
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
String str = toString(i % mod_to);
|
||||
column_string->insertData(str.data(), str.size());
|
||||
|
||||
if (ref_map.count(str) == 0)
|
||||
ref_map[str] = ref_map.size();
|
||||
|
||||
indexes[i]= ref_map[str];
|
||||
}
|
||||
|
||||
auto idx = column_unique->uniqueInsertRangeFrom(*column_string, 0, num_values);
|
||||
ASSERT_EQ(idx->size(), num_values);
|
||||
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
ASSERT_EQ(indexes[i] + 1, idx->getUInt(i)) << "Different indexes at position " << i;
|
||||
}
|
||||
|
||||
auto & nested = column_unique->getNestedColumn();
|
||||
ASSERT_EQ(nested->size(), mod_to + 1);
|
||||
|
||||
for (size_t i = 0; i < mod_to; ++i)
|
||||
{
|
||||
ASSERT_EQ(std::to_string(i), nested->getDataAt(i + 1).toString());
|
||||
}
|
||||
}
|
||||
|
||||
TEST(column_unique, column_unique_unique_insert_range_with_overflow_Test)
|
||||
{
|
||||
std::unordered_map<String, size_t> ref_map;
|
||||
auto data_type = std::make_shared<DataTypeString>();
|
||||
auto column_unique = ColumnUnique<ColumnString>::create(*data_type);
|
||||
auto column_string = ColumnString::create();
|
||||
|
||||
size_t num_values = 1000000;
|
||||
size_t mod_to = 1000;
|
||||
|
||||
std::vector<size_t> indexes(num_values);
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
String str = toString(i % mod_to);
|
||||
column_string->insertData(str.data(), str.size());
|
||||
|
||||
if (ref_map.count(str) == 0)
|
||||
ref_map[str] = ref_map.size();
|
||||
|
||||
indexes[i]= ref_map[str];
|
||||
}
|
||||
|
||||
size_t max_val = mod_to / 2;
|
||||
size_t max_dict_size = max_val + 1;
|
||||
auto idx_with_overflow = column_unique->uniqueInsertRangeWithOverflow(*column_string, 0, num_values, max_dict_size);
|
||||
auto & idx = idx_with_overflow.indexes;
|
||||
auto & add_keys = idx_with_overflow.overflowed_keys;
|
||||
|
||||
ASSERT_EQ(idx->size(), num_values);
|
||||
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
ASSERT_EQ(indexes[i] + 1, idx->getUInt(i)) << "Different indexes at position " << i;
|
||||
}
|
||||
|
||||
auto & nested = column_unique->getNestedColumn();
|
||||
ASSERT_EQ(nested->size(), max_dict_size);
|
||||
ASSERT_EQ(add_keys->size(), mod_to - max_val);
|
||||
|
||||
for (size_t i = 0; i < max_val; ++i)
|
||||
{
|
||||
ASSERT_EQ(std::to_string(i), nested->getDataAt(i + 1).toString());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < mod_to - max_val; ++i)
|
||||
{
|
||||
ASSERT_EQ(std::to_string(max_val + i), add_keys->getDataAt(i).toString());
|
||||
}
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
void column_unique_unique_deserialize_from_arena_impl(ColumnType & column, const IDataType & data_type)
|
||||
{
|
||||
size_t num_values = column.size();
|
||||
|
||||
{
|
||||
/// Check serialization is reversible.
|
||||
Arena arena;
|
||||
auto column_unique_pattern = ColumnUnique<ColumnString>::create(data_type);
|
||||
auto column_unique = ColumnUnique<ColumnString>::create(data_type);
|
||||
auto idx = column_unique_pattern->uniqueInsertRangeFrom(column, 0, num_values);
|
||||
|
||||
const char * pos = nullptr;
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
auto ref = column_unique_pattern->serializeValueIntoArena(idx->getUInt(i), arena, pos);
|
||||
const char * new_pos;
|
||||
column_unique->uniqueDeserializeAndInsertFromArena(ref.data, new_pos);
|
||||
ASSERT_EQ(new_pos - ref.data, ref.size) << "Deserialized data has different sizes at position " << i;
|
||||
|
||||
ASSERT_EQ(column_unique_pattern->getNestedNotNullableColumn()->getDataAt(idx->getUInt(i)),
|
||||
column_unique->getNestedNotNullableColumn()->getDataAt(idx->getUInt(i)))
|
||||
<< "Deserialized data is different from pattern at position " << i;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
/// Check serialization the same with ordinary column.
|
||||
Arena arena_string;
|
||||
Arena arena_lc;
|
||||
auto column_unique = ColumnUnique<ColumnString>::create(data_type);
|
||||
auto idx = column_unique->uniqueInsertRangeFrom(column, 0, num_values);
|
||||
|
||||
const char * pos_string = nullptr;
|
||||
const char * pos_lc = nullptr;
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
auto ref_string = column.serializeValueIntoArena(i, arena_string, pos_string);
|
||||
auto ref_lc = column_unique->serializeValueIntoArena(idx->getUInt(i), arena_lc, pos_lc);
|
||||
ASSERT_EQ(ref_string, ref_lc) << "Serialized data is different from pattern at position " << i;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST(column_unique, column_unique_unique_deserialize_from_arena_String_Test)
|
||||
{
|
||||
auto data_type = std::make_shared<DataTypeString>();
|
||||
auto column_string = ColumnString::create();
|
||||
|
||||
size_t num_values = 1000000;
|
||||
size_t mod_to = 1000;
|
||||
|
||||
std::vector<size_t> indexes(num_values);
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
String str = toString(i % mod_to);
|
||||
column_string->insertData(str.data(), str.size());
|
||||
}
|
||||
|
||||
column_unique_unique_deserialize_from_arena_impl(*column_string, *data_type);
|
||||
}
|
||||
|
||||
TEST(column_unique, column_unique_unique_deserialize_from_arena_Nullable_String_Test)
|
||||
{
|
||||
auto data_type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>());
|
||||
auto column_string = ColumnString::create();
|
||||
auto null_mask = ColumnUInt8::create();
|
||||
|
||||
size_t num_values = 1000000;
|
||||
size_t mod_to = 1000;
|
||||
|
||||
std::vector<size_t> indexes(num_values);
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
String str = toString(i % mod_to);
|
||||
column_string->insertData(str.data(), str.size());
|
||||
|
||||
null_mask->insertValue(i % 3 ? 1 : 0);
|
||||
}
|
||||
|
||||
auto column = ColumnNullable::create(std::move(column_string), std::move(null_mask));
|
||||
column_unique_unique_deserialize_from_arena_impl(*column, *data_type);
|
||||
}
|
@ -48,6 +48,7 @@ protected:
|
||||
#endif
|
||||
|
||||
/** Allocator with optimization to place small memory ranges in automatic memory.
|
||||
* TODO alignment
|
||||
*/
|
||||
template <typename Base, size_t N = 64>
|
||||
class AllocatorWithStackMemory : private Base
|
||||
|
@ -38,6 +38,7 @@ public:
|
||||
};
|
||||
|
||||
|
||||
/// Allocates in Arena with proper alignment.
|
||||
template <size_t alignment>
|
||||
class AlignedArenaAllocator
|
||||
{
|
||||
@ -69,14 +70,14 @@ public:
|
||||
|
||||
|
||||
/// Switches to ordinary Allocator after REAL_ALLOCATION_TRESHOLD bytes to avoid fragmentation and trash in Arena.
|
||||
template <size_t REAL_ALLOCATION_TRESHOLD = 4096, typename TRealAllocator = Allocator<false>, typename TArenaAllocator = ArenaAllocator>
|
||||
template <size_t REAL_ALLOCATION_TRESHOLD = 4096, typename TRealAllocator = Allocator<false>, typename TArenaAllocator = ArenaAllocator, size_t alignment = 0>
|
||||
class MixedArenaAllocator : private TRealAllocator
|
||||
{
|
||||
public:
|
||||
|
||||
void * alloc(size_t size, Arena * arena)
|
||||
{
|
||||
return (size < REAL_ALLOCATION_TRESHOLD) ? TArenaAllocator::alloc(size, arena) : TRealAllocator::alloc(size);
|
||||
return (size < REAL_ALLOCATION_TRESHOLD) ? TArenaAllocator::alloc(size, arena) : TRealAllocator::alloc(size, alignment);
|
||||
}
|
||||
|
||||
void * realloc(void * buf, size_t old_size, size_t new_size, Arena * arena)
|
||||
@ -87,9 +88,9 @@ public:
|
||||
return TArenaAllocator::realloc(buf, old_size, new_size, arena);
|
||||
|
||||
if (old_size >= REAL_ALLOCATION_TRESHOLD)
|
||||
return TRealAllocator::realloc(buf, old_size, new_size);
|
||||
return TRealAllocator::realloc(buf, old_size, new_size, alignment);
|
||||
|
||||
void * new_buf = TRealAllocator::alloc(new_size);
|
||||
void * new_buf = TRealAllocator::alloc(new_size, alignment);
|
||||
memcpy(new_buf, buf, old_size);
|
||||
return new_buf;
|
||||
}
|
||||
@ -103,11 +104,11 @@ public:
|
||||
|
||||
|
||||
template <size_t alignment, size_t REAL_ALLOCATION_TRESHOLD = 4096>
|
||||
using MixedAlignedArenaAllocator = MixedArenaAllocator<REAL_ALLOCATION_TRESHOLD, Allocator<false>, AlignedArenaAllocator<alignment>>;
|
||||
using MixedAlignedArenaAllocator = MixedArenaAllocator<REAL_ALLOCATION_TRESHOLD, Allocator<false>, AlignedArenaAllocator<alignment>, alignment>;
|
||||
|
||||
|
||||
template <size_t N = 64, typename Base = ArenaAllocator>
|
||||
class ArenaAllocatorWithStackMemoty : public Base
|
||||
class ArenaAllocatorWithStackMemory : public Base
|
||||
{
|
||||
char stack_memory[N];
|
||||
|
||||
|
@ -658,12 +658,8 @@ protected:
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
/// Only for non-zero keys. Find the right place, insert the key there, if it does not already exist. Set iterator to the cell in output parameter.
|
||||
void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value)
|
||||
void ALWAYS_INLINE emplaceNonZeroImpl(size_t place_value, Key x, iterator & it, bool & inserted, size_t hash_value)
|
||||
{
|
||||
size_t place_value = findCell(x, hash_value, grower.place(hash_value));
|
||||
|
||||
it = iterator(this, &buf[place_value]);
|
||||
|
||||
if (!buf[place_value].isZero(*this))
|
||||
@ -698,6 +694,21 @@ protected:
|
||||
}
|
||||
}
|
||||
|
||||
/// Only for non-zero keys. Find the right place, insert the key there, if it does not already exist. Set iterator to the cell in output parameter.
|
||||
void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value)
|
||||
{
|
||||
size_t place_value = findCell(x, hash_value, grower.place(hash_value));
|
||||
emplaceNonZeroImpl(place_value, x, it, inserted, hash_value);
|
||||
}
|
||||
|
||||
/// Same but find place using object. Hack for ReverseIndex.
|
||||
template <typename ObjectToCompareWith>
|
||||
void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
|
||||
{
|
||||
size_t place_value = findCell(object, hash_value, grower.place(hash_value));
|
||||
emplaceNonZeroImpl(place_value, x, it, inserted, hash_value);
|
||||
}
|
||||
|
||||
|
||||
public:
|
||||
/// Insert a value. In the case of any more complex values, it is better to use the `emplace` function.
|
||||
@ -753,6 +764,13 @@ public:
|
||||
emplaceNonZero(x, it, inserted, hash_value);
|
||||
}
|
||||
|
||||
/// Same, but search position by object. Hack for ReverseIndex.
|
||||
template <typename ObjectToCompareWith>
|
||||
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
|
||||
{
|
||||
if (!emplaceIfZero(x, it, inserted, hash_value))
|
||||
emplaceNonZero(x, it, inserted, hash_value, object);
|
||||
}
|
||||
|
||||
/// Copy the cell from another hash table. It is assumed that the cell is not zero, and also that there was no such key in the table yet.
|
||||
void ALWAYS_INLINE insertUniqueNonZero(const Cell * cell, size_t hash_value)
|
||||
|
@ -45,6 +45,9 @@ namespace DB
|
||||
* Because sometimes we have many small objects, that share same allocator with same parameters,
|
||||
* and we must avoid larger object size due to storing the same parameters in each object.
|
||||
* This is required for states of aggregate functions.
|
||||
*
|
||||
* TODO Pass alignment to Allocator.
|
||||
* TODO Allow greater alignment than alignof(T). Example: array of char aligned to page size.
|
||||
*/
|
||||
template <typename T, size_t INITIAL_SIZE = 4096, typename TAllocator = Allocator<false>, size_t pad_right_ = 0>
|
||||
class PODArray : private boost::noncopyable, private TAllocator /// empty base optimization
|
||||
|
@ -61,8 +61,6 @@
|
||||
/// The boundary on which the blocks for asynchronous file operations should be aligned.
|
||||
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096
|
||||
|
||||
#define DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS 7500
|
||||
|
||||
#define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800
|
||||
#define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1
|
||||
/// Maximum namber of http-connections between two endpoints
|
||||
|
@ -27,35 +27,43 @@ void writeODBCString(WriteBuffer & out, const std::string & str)
|
||||
out.write(str.data(), str.size());
|
||||
}
|
||||
|
||||
void ODBCDriver2BlockOutputStream::write(const Block & block)
|
||||
static void writeRow(const Block & block, size_t row_idx, WriteBuffer & out, const FormatSettings & format_settings, std::string & buffer)
|
||||
{
|
||||
const size_t rows = block.rows();
|
||||
const size_t columns = block.columns();
|
||||
String text_value;
|
||||
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
size_t columns = block.columns();
|
||||
for (size_t column_idx = 0; column_idx < columns; ++column_idx)
|
||||
{
|
||||
for (size_t j = 0; j < columns; ++j)
|
||||
{
|
||||
text_value.resize(0);
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(j);
|
||||
buffer.clear();
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(column_idx);
|
||||
|
||||
if (col.column->isNullAt(i))
|
||||
if (col.column->isNullAt(row_idx))
|
||||
{
|
||||
writeIntBinary(Int32(-1), out);
|
||||
}
|
||||
else
|
||||
{
|
||||
{
|
||||
writeIntBinary(Int32(-1), out);
|
||||
}
|
||||
else
|
||||
{
|
||||
{
|
||||
WriteBufferFromString text_out(text_value);
|
||||
col.type->serializeText(*col.column, i, text_out, format_settings);
|
||||
}
|
||||
writeODBCString(out, text_value);
|
||||
WriteBufferFromString text_out(buffer);
|
||||
col.type->serializeText(*col.column, row_idx, text_out, format_settings);
|
||||
}
|
||||
writeODBCString(out, buffer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void ODBCDriver2BlockOutputStream::write(const Block & block)
|
||||
{
|
||||
String text_value;
|
||||
const size_t rows = block.rows();
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
writeRow(block, i, out, format_settings, text_value);
|
||||
}
|
||||
|
||||
void ODBCDriver2BlockOutputStream::writeSuffix()
|
||||
{
|
||||
if (totals)
|
||||
write(totals);
|
||||
}
|
||||
|
||||
void ODBCDriver2BlockOutputStream::writePrefix()
|
||||
{
|
||||
const size_t columns = header.columns();
|
||||
|
@ -17,7 +17,7 @@ class WriteBuffer;
|
||||
* A header is displayed with the required information.
|
||||
* The data is then output in the order of the rows. Each value is displayed as follows: length in Int32 format (-1 for NULL), then data in text form.
|
||||
*/
|
||||
class ODBCDriver2BlockOutputStream : public IBlockOutputStream
|
||||
class ODBCDriver2BlockOutputStream final : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
ODBCDriver2BlockOutputStream(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings);
|
||||
@ -28,17 +28,24 @@ public:
|
||||
}
|
||||
void write(const Block & block) override;
|
||||
void writePrefix() override;
|
||||
void writeSuffix() override;
|
||||
|
||||
void flush() override;
|
||||
std::string getContentType() const override
|
||||
{
|
||||
return "application/octet-stream";
|
||||
}
|
||||
void setTotals(const Block & totals_) override { totals = totals_; }
|
||||
|
||||
private:
|
||||
WriteBuffer & out;
|
||||
const Block header;
|
||||
const FormatSettings format_settings;
|
||||
|
||||
protected:
|
||||
Block totals;
|
||||
};
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
@ -178,7 +178,8 @@ struct AggregationMethodOneNumber
|
||||
*/
|
||||
void init(ColumnRawPtrs & key_columns)
|
||||
{
|
||||
vec = &static_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData()[0];
|
||||
/// We may interpret ColumnInt32 as ColumnUInt32. This breaks strict aliasing but compiler doesn't see it.
|
||||
vec = &reinterpret_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData()[0];
|
||||
}
|
||||
|
||||
/// Get the key from the key columns for insertion into the hash table.
|
||||
|
@ -48,7 +48,7 @@ inline bool isLocal(const Cluster::Address & address, const Poco::Net::SocketAdd
|
||||
|
||||
/// Implementation of Cluster::Address class
|
||||
|
||||
Cluster::Address::Address(Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
{
|
||||
UInt16 clickhouse_port = static_cast<UInt16>(config.getInt("tcp_port", 0));
|
||||
|
||||
@ -125,7 +125,7 @@ String Cluster::Address::toStringFull() const
|
||||
|
||||
/// Implementation of Clusters class
|
||||
|
||||
Clusters::Clusters(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name)
|
||||
Clusters::Clusters(const Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name)
|
||||
{
|
||||
updateClusters(config, settings, config_name);
|
||||
}
|
||||
@ -147,7 +147,7 @@ void Clusters::setCluster(const String & cluster_name, const std::shared_ptr<Clu
|
||||
}
|
||||
|
||||
|
||||
void Clusters::updateClusters(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name)
|
||||
void Clusters::updateClusters(const Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(config_name, config_keys);
|
||||
@ -174,7 +174,7 @@ Clusters::Impl Clusters::getContainer() const
|
||||
|
||||
/// Implementation of `Cluster` class
|
||||
|
||||
Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & cluster_name)
|
||||
Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & cluster_name)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(cluster_name, config_keys);
|
||||
|
@ -16,7 +16,7 @@ namespace DB
|
||||
class Cluster
|
||||
{
|
||||
public:
|
||||
Cluster(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & cluster_name);
|
||||
Cluster(const Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & cluster_name);
|
||||
|
||||
/// Construct a cluster by the names of shards and replicas.
|
||||
/// Local are treated as well as remote ones if treat_local_as_remote is true.
|
||||
@ -66,7 +66,7 @@ public:
|
||||
Protocol::Secure secure = Protocol::Secure::Disable;
|
||||
|
||||
Address() = default;
|
||||
Address(Poco::Util::AbstractConfiguration & config, const String & config_prefix);
|
||||
Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
|
||||
Address(const String & host_port_, const String & user_, const String & password_, UInt16 clickhouse_port);
|
||||
|
||||
/// Returns 'escaped_host_name:port'
|
||||
@ -178,7 +178,7 @@ using ClusterPtr = std::shared_ptr<Cluster>;
|
||||
class Clusters
|
||||
{
|
||||
public:
|
||||
Clusters(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name = "remote_servers");
|
||||
Clusters(const Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name = "remote_servers");
|
||||
|
||||
Clusters(const Clusters &) = delete;
|
||||
Clusters & operator=(const Clusters &) = delete;
|
||||
@ -186,7 +186,7 @@ public:
|
||||
ClusterPtr getCluster(const std::string & cluster_name) const;
|
||||
void setCluster(const String & cluster_name, const ClusterPtr & cluster);
|
||||
|
||||
void updateClusters(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name);
|
||||
void updateClusters(const Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name);
|
||||
|
||||
public:
|
||||
using Impl = std::map<String, ClusterPtr>;
|
||||
|
@ -157,10 +157,10 @@ struct ContextShared
|
||||
public:
|
||||
size_t operator()(const Context::SessionKey & key) const
|
||||
{
|
||||
size_t seed = 0;
|
||||
boost::hash_combine(seed, key.first);
|
||||
boost::hash_combine(seed, key.second);
|
||||
return seed;
|
||||
SipHash hash;
|
||||
hash.update(key.first);
|
||||
hash.update(key.second);
|
||||
return hash.get64();
|
||||
}
|
||||
};
|
||||
|
||||
@ -549,7 +549,7 @@ void Context::setConfig(const ConfigurationPtr & config)
|
||||
shared->config = config;
|
||||
}
|
||||
|
||||
Poco::Util::AbstractConfiguration & Context::getConfigRef() const
|
||||
const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
return shared->config ? *shared->config : Poco::Util::Application::instance().config();
|
||||
@ -1537,95 +1537,50 @@ Compiler & Context::getCompiler()
|
||||
void Context::initializeSystemLogs()
|
||||
{
|
||||
auto lock = getLock();
|
||||
system_logs = std::make_shared<SystemLogs>();
|
||||
|
||||
if (!global_context)
|
||||
throw Exception("Logical error: no global context for system logs", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
system_logs = std::make_shared<SystemLogs>(*global_context, getConfigRef());
|
||||
}
|
||||
|
||||
|
||||
QueryLog * Context::getQueryLog(bool create_if_not_exists)
|
||||
QueryLog * Context::getQueryLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (!system_logs)
|
||||
if (!system_logs || !system_logs->query_log)
|
||||
return nullptr;
|
||||
|
||||
if (!system_logs->query_log)
|
||||
{
|
||||
if (!create_if_not_exists)
|
||||
return nullptr;
|
||||
|
||||
if (shared->shutdown_called)
|
||||
throw Exception("Logical error: query log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!global_context)
|
||||
throw Exception("Logical error: no global context for query log", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
system_logs->query_log = createDefaultSystemLog<QueryLog>(*global_context, "system", "query_log", getConfigRef(), "query_log");
|
||||
}
|
||||
|
||||
return system_logs->query_log.get();
|
||||
}
|
||||
|
||||
|
||||
QueryThreadLog * Context::getQueryThreadLog(bool create_if_not_exists)
|
||||
QueryThreadLog * Context::getQueryThreadLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (!system_logs)
|
||||
if (!system_logs || !system_logs->query_thread_log)
|
||||
return nullptr;
|
||||
|
||||
if (!system_logs->query_thread_log)
|
||||
{
|
||||
if (!create_if_not_exists)
|
||||
return nullptr;
|
||||
|
||||
if (shared->shutdown_called)
|
||||
throw Exception("Logical error: query log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!global_context)
|
||||
throw Exception("Logical error: no global context for query thread log", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
system_logs->query_thread_log = createDefaultSystemLog<QueryThreadLog>(
|
||||
*global_context, "system", "query_thread_log", getConfigRef(), "query_thread_log");
|
||||
}
|
||||
|
||||
return system_logs->query_thread_log.get();
|
||||
}
|
||||
|
||||
|
||||
PartLog * Context::getPartLog(const String & part_database, bool create_if_not_exists)
|
||||
PartLog * Context::getPartLog(const String & part_database)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
auto & config = getConfigRef();
|
||||
if (!config.has("part_log"))
|
||||
return nullptr;
|
||||
|
||||
/// System logs are shutting down.
|
||||
if (!system_logs)
|
||||
if (!system_logs || !system_logs->part_log)
|
||||
return nullptr;
|
||||
|
||||
String database = config.getString("part_log.database", "system");
|
||||
|
||||
/// Will not log operations on system tables (including part_log itself).
|
||||
/// It doesn't make sense and not allow to destruct PartLog correctly due to infinite logging and flushing,
|
||||
/// and also make troubles on startup.
|
||||
if (!part_database.empty() && part_database == database)
|
||||
if (part_database == system_logs->part_log_database)
|
||||
return nullptr;
|
||||
|
||||
if (!system_logs->part_log)
|
||||
{
|
||||
if (!create_if_not_exists)
|
||||
return nullptr;
|
||||
|
||||
if (shared->shutdown_called)
|
||||
throw Exception("Logical error: part log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!global_context)
|
||||
throw Exception("Logical error: no global context for part log", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
system_logs->part_log = createDefaultSystemLog<PartLog>(*global_context, "system", "part_log", getConfigRef(), "part_log");
|
||||
}
|
||||
|
||||
return system_logs->part_log.get();
|
||||
}
|
||||
|
||||
|
@ -168,7 +168,7 @@ public:
|
||||
|
||||
/// Global application configuration settings.
|
||||
void setConfig(const ConfigurationPtr & config);
|
||||
Poco::Util::AbstractConfiguration & getConfigRef() const;
|
||||
const Poco::Util::AbstractConfiguration & getConfigRef() const;
|
||||
|
||||
/** Take the list of users, quotas and configuration profiles from this config.
|
||||
* The list of users is completely replaced.
|
||||
@ -389,12 +389,12 @@ public:
|
||||
void initializeSystemLogs();
|
||||
|
||||
/// Nullptr if the query log is not ready for this moment.
|
||||
QueryLog * getQueryLog(bool create_if_not_exists = true);
|
||||
QueryThreadLog * getQueryThreadLog(bool create_if_not_exists = true);
|
||||
QueryLog * getQueryLog();
|
||||
QueryThreadLog * getQueryThreadLog();
|
||||
|
||||
/// Returns an object used to log opertaions with parts if it possible.
|
||||
/// Provide table name to make required cheks.
|
||||
PartLog * getPartLog(const String & part_database, bool create_if_not_exists = true);
|
||||
PartLog * getPartLog(const String & part_database);
|
||||
|
||||
const MergeTreeSettings & getMergeTreeSettings() const;
|
||||
|
||||
|
@ -143,8 +143,15 @@ ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_column
|
||||
a.type = ARRAY_JOIN;
|
||||
a.array_joined_columns = array_joined_columns;
|
||||
a.array_join_is_left = array_join_is_left;
|
||||
a.unaligned_array_join = context.getSettingsRef().enable_unaligned_array_join;
|
||||
|
||||
if (array_join_is_left)
|
||||
if (a.unaligned_array_join)
|
||||
{
|
||||
a.function_length = FunctionFactory::instance().get("length", context);
|
||||
a.function_greatest = FunctionFactory::instance().get("greatest", context);
|
||||
a.function_arrayResize = FunctionFactory::instance().get("arrayResize", context);
|
||||
}
|
||||
else if (array_join_is_left)
|
||||
a.function_builder = FunctionFactory::instance().get("emptyArrayToSingle", context);
|
||||
|
||||
return a;
|
||||
@ -378,7 +385,44 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||
|
||||
/// If LEFT ARRAY JOIN, then we create columns in which empty arrays are replaced by arrays with one element - the default value.
|
||||
std::map<String, ColumnPtr> non_empty_array_columns;
|
||||
if (array_join_is_left)
|
||||
|
||||
if (unaligned_array_join)
|
||||
{
|
||||
/// Resize all array joined columns to the longest one, (at least 1 if LEFT ARRAY JOIN), padded with default values.
|
||||
auto rows = block.rows();
|
||||
auto uint64 = std::make_shared<DataTypeUInt64>();
|
||||
ColumnWithTypeAndName column_of_max_length;
|
||||
if (array_join_is_left)
|
||||
column_of_max_length = ColumnWithTypeAndName(uint64->createColumnConst(rows, 1u), uint64, {});
|
||||
else
|
||||
column_of_max_length = ColumnWithTypeAndName(uint64->createColumnConst(rows, 0u), uint64, {});
|
||||
|
||||
for (const auto & name : array_joined_columns)
|
||||
{
|
||||
auto & src_col = block.getByName(name);
|
||||
|
||||
Block tmp_block{src_col, {{}, uint64, {}}};
|
||||
function_length->build({src_col})->execute(tmp_block, {0}, 1, rows);
|
||||
|
||||
Block tmp_block2{
|
||||
column_of_max_length, tmp_block.safeGetByPosition(1), {{}, uint64, {}}};
|
||||
function_greatest->build({column_of_max_length, tmp_block.safeGetByPosition(1)})->execute(tmp_block2, {0, 1}, 2, rows);
|
||||
column_of_max_length = tmp_block2.safeGetByPosition(2);
|
||||
}
|
||||
|
||||
for (const auto & name : array_joined_columns)
|
||||
{
|
||||
auto & src_col = block.getByName(name);
|
||||
|
||||
Block tmp_block{src_col, column_of_max_length, {{}, src_col.type, {}}};
|
||||
function_arrayResize->build({src_col, column_of_max_length})->execute(tmp_block, {0, 1}, 2, rows);
|
||||
any_array_ptr = src_col.column = tmp_block.safeGetByPosition(2).column;
|
||||
}
|
||||
if (ColumnPtr converted = any_array_ptr->convertToFullColumnIfConst())
|
||||
any_array_ptr = converted;
|
||||
any_array = typeid_cast<const ColumnArray *>(&*any_array_ptr);
|
||||
}
|
||||
else if (array_join_is_left && !unaligned_array_join)
|
||||
{
|
||||
for (const auto & name : array_joined_columns)
|
||||
{
|
||||
@ -407,13 +451,13 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||
if (!typeid_cast<const DataTypeArray *>(&*current.type))
|
||||
throw Exception("ARRAY JOIN of not array: " + current.name, ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
ColumnPtr array_ptr = array_join_is_left ? non_empty_array_columns[current.name] : current.column;
|
||||
ColumnPtr array_ptr = (array_join_is_left && !unaligned_array_join) ? non_empty_array_columns[current.name] : current.column;
|
||||
|
||||
if (ColumnPtr converted = array_ptr->convertToFullColumnIfConst())
|
||||
array_ptr = converted;
|
||||
|
||||
const ColumnArray & array = typeid_cast<const ColumnArray &>(*array_ptr);
|
||||
if (!array.hasEqualOffsets(typeid_cast<const ColumnArray &>(*any_array_ptr)))
|
||||
if (!unaligned_array_join && !array.hasEqualOffsets(typeid_cast<const ColumnArray &>(*any_array_ptr)))
|
||||
throw Exception("Sizes of ARRAY-JOIN-ed arrays do not match", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
|
||||
current.column = typeid_cast<const ColumnArray &>(*array_ptr).getDataPtr();
|
||||
|
@ -87,6 +87,12 @@ public:
|
||||
/// For APPLY_FUNCTION and LEFT ARRAY JOIN.
|
||||
/// FunctionBuilder is used before action was added to ExpressionActions (when we don't know types of arguments).
|
||||
FunctionBuilderPtr function_builder;
|
||||
|
||||
/// For unaligned [LEFT] ARRAY JOIN
|
||||
FunctionBuilderPtr function_length;
|
||||
FunctionBuilderPtr function_greatest;
|
||||
FunctionBuilderPtr function_arrayResize;
|
||||
|
||||
/// Can be used after action was added to ExpressionActions if we want to get function signature or properties like monotonicity.
|
||||
FunctionBasePtr function_base;
|
||||
/// Prepared function which is used in function execution.
|
||||
@ -97,6 +103,7 @@ public:
|
||||
/// For ARRAY_JOIN
|
||||
NameSet array_joined_columns;
|
||||
bool array_join_is_left = false;
|
||||
bool unaligned_array_join = false;
|
||||
|
||||
/// For JOIN
|
||||
std::shared_ptr<const Join> join;
|
||||
|
@ -58,10 +58,6 @@ struct ExpressionAnalyzerData
|
||||
|
||||
bool has_global_subqueries = false;
|
||||
|
||||
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.
|
||||
/// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v".
|
||||
NameToNameMap array_join_result_to_source;
|
||||
|
||||
/// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries.
|
||||
Tables external_tables;
|
||||
|
||||
|
@ -15,7 +15,7 @@ class ISecurityManager
|
||||
public:
|
||||
using UserPtr = std::shared_ptr<const User>;
|
||||
|
||||
virtual void loadFromConfig(Poco::Util::AbstractConfiguration & config) = 0;
|
||||
virtual void loadFromConfig(const Poco::Util::AbstractConfiguration & config) = 0;
|
||||
|
||||
/// Find user and make authorize checks
|
||||
virtual UserPtr authorizeAndGetUser(
|
||||
|
@ -606,6 +606,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
|
||||
executeRollupOrCube(pipeline, Modificator::ROLLUP);
|
||||
else if (query.group_by_with_cube)
|
||||
executeRollupOrCube(pipeline, Modificator::CUBE);
|
||||
if ((query.group_by_with_rollup || query.group_by_with_cube) && expressions.has_having)
|
||||
executeHaving(pipeline, expressions.before_having);
|
||||
}
|
||||
else if (expressions.has_having)
|
||||
executeHaving(pipeline, expressions.before_having);
|
||||
@ -625,10 +627,15 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
|
||||
executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, final);
|
||||
}
|
||||
|
||||
if (query.group_by_with_rollup && !aggregate_final)
|
||||
executeRollupOrCube(pipeline, Modificator::ROLLUP);
|
||||
else if (query.group_by_with_cube && !aggregate_final)
|
||||
executeRollupOrCube(pipeline, Modificator::CUBE);
|
||||
if ((query.group_by_with_rollup || query.group_by_with_cube) && !aggregate_final)
|
||||
{
|
||||
if (query.group_by_with_rollup)
|
||||
executeRollupOrCube(pipeline, Modificator::ROLLUP);
|
||||
else if (query.group_by_with_cube)
|
||||
executeRollupOrCube(pipeline, Modificator::CUBE);
|
||||
if (expressions.has_having)
|
||||
executeHaving(pipeline, expressions.before_having);
|
||||
}
|
||||
}
|
||||
|
||||
if (expressions.has_order_by)
|
||||
@ -1483,3 +1490,4 @@ void InterpreterSelectQuery::initSettings()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -206,9 +206,9 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
break;
|
||||
case Type::FLUSH_LOGS:
|
||||
executeCommandsAndThrowIfError(
|
||||
[&] () { if (auto query_log = context.getQueryLog(false)) query_log->flush(); },
|
||||
[&] () { if (auto part_log = context.getPartLog("", false)) part_log->flush(); },
|
||||
[&] () { if (auto query_thread_log = context.getQueryThreadLog(false)) query_thread_log->flush(); }
|
||||
[&] () { if (auto query_log = context.getQueryLog()) query_log->flush(); },
|
||||
[&] () { if (auto part_log = context.getPartLog("")) part_log->flush(); },
|
||||
[&] () { if (auto query_thread_log = context.getQueryThreadLog()) query_thread_log->flush(); }
|
||||
);
|
||||
break;
|
||||
case Type::STOP_LISTEN_QUERIES:
|
||||
|
@ -35,7 +35,8 @@ struct JoinKeyGetterOneNumber
|
||||
*/
|
||||
JoinKeyGetterOneNumber(const ColumnRawPtrs & key_columns)
|
||||
{
|
||||
vec = &static_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData()[0];
|
||||
/// We may interpret ColumnInt32 as ColumnUInt32. This breaks strict aliasing but compiler doesn't see it.
|
||||
vec = &reinterpret_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData()[0];
|
||||
}
|
||||
|
||||
Key getKey(
|
||||
|
@ -23,7 +23,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
template <typename Counter>
|
||||
void QuotaValues<Counter>::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config)
|
||||
void QuotaValues<Counter>::initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
queries = config.getUInt64(config_elem + ".queries", 0);
|
||||
errors = config.getUInt64(config_elem + ".errors", 0);
|
||||
@ -34,11 +34,12 @@ void QuotaValues<Counter>::initFromConfig(const String & config_elem, Poco::Util
|
||||
execution_time_usec = config.getUInt64(config_elem + ".execution_time", 0) * 1000000ULL;
|
||||
}
|
||||
|
||||
template void QuotaValues<size_t>::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
|
||||
template void QuotaValues<std::atomic<size_t>>::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
|
||||
template void QuotaValues<size_t>::initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
template void QuotaValues<std::atomic<size_t>>::initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
|
||||
void QuotaForInterval::initFromConfig(const String & config_elem, time_t duration_, bool randomize_, time_t offset_, Poco::Util::AbstractConfiguration & config)
|
||||
void QuotaForInterval::initFromConfig(
|
||||
const String & config_elem, time_t duration_, bool randomize_, time_t offset_, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
rounded_time.store(0, std::memory_order_relaxed);
|
||||
duration = duration_;
|
||||
@ -160,7 +161,7 @@ void QuotaForInterval::check(
|
||||
}
|
||||
|
||||
|
||||
void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, pcg64 & rng)
|
||||
void QuotaForIntervals::initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, pcg64 & rng)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(config_elem, config_keys);
|
||||
@ -251,7 +252,7 @@ String QuotaForIntervals::toString() const
|
||||
}
|
||||
|
||||
|
||||
void Quota::loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, pcg64 & rng)
|
||||
void Quota::loadFromConfig(const String & config_elem, const String & name_, const Poco::Util::AbstractConfiguration & config, pcg64 & rng)
|
||||
{
|
||||
name = name_;
|
||||
|
||||
@ -307,7 +308,7 @@ QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_na
|
||||
}
|
||||
|
||||
|
||||
void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config)
|
||||
void Quotas::loadFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
pcg64 rng;
|
||||
|
||||
|
@ -63,7 +63,7 @@ struct QuotaValues
|
||||
tuple() = std::make_tuple(0, 0, 0, 0, 0, 0, 0);
|
||||
}
|
||||
|
||||
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
|
||||
void initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
bool operator== (const QuotaValues & rhs) const
|
||||
{
|
||||
@ -109,7 +109,7 @@ struct QuotaForInterval
|
||||
QuotaForInterval() = default;
|
||||
QuotaForInterval(time_t duration_) : duration(duration_) {}
|
||||
|
||||
void initFromConfig(const String & config_elem, time_t duration_, bool randomize_, time_t offset_, Poco::Util::AbstractConfiguration & config);
|
||||
void initFromConfig(const String & config_elem, time_t duration_, bool randomize_, time_t offset_, const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
/// Increase current value.
|
||||
void addQuery() noexcept;
|
||||
@ -191,7 +191,7 @@ public:
|
||||
return cont.empty();
|
||||
}
|
||||
|
||||
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, pcg64 & rng);
|
||||
void initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, pcg64 & rng);
|
||||
|
||||
/// Set maximum values (limits) from passed argument.
|
||||
/// Remove intervals that does not exist in argument. Add intervals from argument, that we don't have.
|
||||
@ -241,7 +241,7 @@ struct Quota
|
||||
|
||||
bool keyed_by_ip = false;
|
||||
|
||||
void loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, pcg64 & rng);
|
||||
void loadFromConfig(const String & config_elem, const String & name_, const Poco::Util::AbstractConfiguration & config, pcg64 & rng);
|
||||
QuotaForIntervalsPtr get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip);
|
||||
};
|
||||
|
||||
@ -254,7 +254,7 @@ private:
|
||||
Container cont;
|
||||
|
||||
public:
|
||||
void loadFromConfig(Poco::Util::AbstractConfiguration & config);
|
||||
void loadFromConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
QuotaForIntervalsPtr get(const String & name, const String & quota_key,
|
||||
const String & user_name, const Poco::Net::IPAddress & ip);
|
||||
};
|
||||
|
@ -29,7 +29,7 @@ namespace ErrorCodes
|
||||
|
||||
using UserPtr = SecurityManager::UserPtr;
|
||||
|
||||
void SecurityManager::loadFromConfig(Poco::Util::AbstractConfiguration & config)
|
||||
void SecurityManager::loadFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
Container new_users;
|
||||
|
||||
|
@ -17,7 +17,7 @@ private:
|
||||
Container users;
|
||||
|
||||
public:
|
||||
void loadFromConfig(Poco::Util::AbstractConfiguration & config) override;
|
||||
void loadFromConfig(const Poco::Util::AbstractConfiguration & config) override;
|
||||
|
||||
UserPtr authorizeAndGetUser(
|
||||
const String & user_name,
|
||||
|
@ -37,7 +37,8 @@ struct SetMethodOneNumber
|
||||
*/
|
||||
void init(const ColumnRawPtrs & key_columns)
|
||||
{
|
||||
vec = static_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData().data();
|
||||
/// We may interpret ColumnInt32 as ColumnUInt32. This breaks strict aliasing but compiler doesn't see it.
|
||||
vec = reinterpret_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData().data();
|
||||
}
|
||||
|
||||
/// Get key from key columns for insertion into hash table.
|
||||
|
@ -105,7 +105,7 @@ bool Settings::tryGet(const String & name, String & value) const
|
||||
/** Set the settings from the profile (in the server configuration, many settings can be listed in one profile).
|
||||
* The profile can also be set using the `set` functions, like the `profile` setting.
|
||||
*/
|
||||
void Settings::setProfile(const String & profile_name, Poco::Util::AbstractConfiguration & config)
|
||||
void Settings::setProfile(const String & profile_name, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
String elem = "profiles." + profile_name;
|
||||
|
||||
|
@ -188,7 +188,7 @@ struct Settings
|
||||
M(SettingBool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.") \
|
||||
M(SettingUInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.") \
|
||||
M(SettingInt64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. Negative value means infinite.") \
|
||||
M(SettingMilliseconds, stream_flush_interval_ms, DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS, "Timeout for flushing data from streaming storages.") \
|
||||
M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.") \
|
||||
M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)") \
|
||||
M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.") \
|
||||
M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.") \
|
||||
@ -293,6 +293,7 @@ struct Settings
|
||||
M(SettingBool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.") \
|
||||
M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.") \
|
||||
M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.") \
|
||||
M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.") \
|
||||
|
||||
|
||||
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
|
||||
@ -322,7 +323,7 @@ struct Settings
|
||||
/** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings).
|
||||
* The profile can also be set using the `set` functions, like the profile setting.
|
||||
*/
|
||||
void setProfile(const String & profile_name, Poco::Util::AbstractConfiguration & config);
|
||||
void setProfile(const String & profile_name, const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
/// Load settings from configuration file, at "path" prefix in configuration.
|
||||
void loadSettingsFromConfig(const String & path, const Poco::Util::AbstractConfiguration & config);
|
||||
|
@ -3,10 +3,22 @@
|
||||
#include <Interpreters/QueryThreadLog.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
query_log = createDefaultSystemLog<QueryLog>(global_context, "system", "query_log", config, "query_log");
|
||||
query_thread_log = createDefaultSystemLog<QueryThreadLog>(global_context, "system", "query_thread_log", config, "query_thread_log");
|
||||
part_log = createDefaultSystemLog<PartLog>(global_context, "system", "part_log", config, "part_log");
|
||||
|
||||
part_log_database = config.getString("part_log.database", "system");
|
||||
}
|
||||
|
||||
|
||||
SystemLogs::~SystemLogs() = default;
|
||||
|
||||
}
|
||||
|
@ -63,11 +63,14 @@ class PartLog;
|
||||
/// because SystemLog destruction makes insert query while flushing data into underlying tables
|
||||
struct SystemLogs
|
||||
{
|
||||
SystemLogs(Context & global_context, const Poco::Util::AbstractConfiguration & config);
|
||||
~SystemLogs();
|
||||
|
||||
std::unique_ptr<QueryLog> query_log; /// Used to log queries.
|
||||
std::unique_ptr<QueryThreadLog> query_thread_log; /// Used to log query threads.
|
||||
std::unique_ptr<PartLog> part_log; /// Used to log operations with parts
|
||||
|
||||
String part_log_database;
|
||||
};
|
||||
|
||||
|
||||
@ -372,23 +375,25 @@ void SystemLog<LogElement>::prepareTable()
|
||||
is_prepared = true;
|
||||
}
|
||||
|
||||
/// Creates a system log with MergeTree engines using parameters from config
|
||||
/// Creates a system log with MergeTree engine using parameters from config
|
||||
template<typename TSystemLog>
|
||||
std::unique_ptr<TSystemLog> createDefaultSystemLog(
|
||||
Context & context_,
|
||||
const String & default_database_name,
|
||||
const String & default_table_name,
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix)
|
||||
Context & context,
|
||||
const String & default_database_name,
|
||||
const String & default_table_name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix)
|
||||
{
|
||||
String database = config.getString(config_prefix + ".database", default_database_name);
|
||||
String table = config.getString(config_prefix + ".table", default_table_name);
|
||||
static constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500;
|
||||
|
||||
String database = config.getString(config_prefix + ".database", default_database_name);
|
||||
String table = config.getString(config_prefix + ".table", default_table_name);
|
||||
String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)");
|
||||
String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024";
|
||||
|
||||
size_t flush_interval_milliseconds = config.getUInt64("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS);
|
||||
size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS);
|
||||
|
||||
return std::make_unique<TSystemLog>(context_, database, table, engine, flush_interval_milliseconds);
|
||||
return std::make_unique<TSystemLog>(context, database, table, engine, flush_interval_milliseconds);
|
||||
}
|
||||
|
||||
|
||||
|
@ -252,7 +252,7 @@ bool AddressPatterns::contains(const Poco::Net::IPAddress & addr) const
|
||||
return false;
|
||||
}
|
||||
|
||||
void AddressPatterns::addFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config)
|
||||
void AddressPatterns::addFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(config_elem, config_keys);
|
||||
@ -276,7 +276,7 @@ void AddressPatterns::addFromConfig(const String & config_elem, Poco::Util::Abst
|
||||
}
|
||||
|
||||
|
||||
User::User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config)
|
||||
User::User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||
: name(name_)
|
||||
{
|
||||
bool has_password = config.has(config_elem + ".password");
|
||||
|
@ -42,7 +42,7 @@ private:
|
||||
|
||||
public:
|
||||
bool contains(const Poco::Net::IPAddress & addr) const;
|
||||
void addFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
|
||||
void addFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
};
|
||||
|
||||
|
||||
@ -65,7 +65,7 @@ struct User
|
||||
using DatabaseSet = std::unordered_set<std::string>;
|
||||
DatabaseSet databases;
|
||||
|
||||
User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config);
|
||||
User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
};
|
||||
|
||||
|
||||
|
@ -58,7 +58,7 @@ private:
|
||||
throw Exception("Unknown compression method " + name, ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
|
||||
}
|
||||
|
||||
Element(Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
Element(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
{
|
||||
min_part_size = config.getUInt64(config_prefix + ".min_part_size", 0);
|
||||
min_part_size_ratio = config.getDouble(config_prefix + ".min_part_size_ratio", 0);
|
||||
@ -80,7 +80,7 @@ private:
|
||||
public:
|
||||
CompressionSettingsSelector() {} /// Always returns the default method.
|
||||
|
||||
CompressionSettingsSelector(Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
CompressionSettingsSelector(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(config_prefix, keys);
|
||||
|
@ -23,9 +23,14 @@ namespace CurrentMetrics
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static constexpr double thread_sleep_seconds = 10;
|
||||
static constexpr double thread_sleep_seconds_random_part = 1.0;
|
||||
|
||||
constexpr double BackgroundProcessingPool::sleep_seconds;
|
||||
constexpr double BackgroundProcessingPool::sleep_seconds_random_part;
|
||||
/// For exponential backoff.
|
||||
static constexpr double task_sleep_seconds_when_no_work_min = 10;
|
||||
static constexpr double task_sleep_seconds_when_no_work_max = 600;
|
||||
static constexpr double task_sleep_seconds_when_no_work_multiplier = 1.1;
|
||||
static constexpr double task_sleep_seconds_when_no_work_random_part = 1.0;
|
||||
|
||||
|
||||
void BackgroundProcessingPoolTaskInfo::wake()
|
||||
@ -137,7 +142,7 @@ void BackgroundProcessingPool::threadFunction()
|
||||
CurrentThread::getMemoryTracker().setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool);
|
||||
|
||||
pcg64 rng(randomSeed());
|
||||
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
|
||||
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, thread_sleep_seconds_random_part)(rng)));
|
||||
|
||||
while (!shutdown)
|
||||
{
|
||||
@ -172,8 +177,8 @@ void BackgroundProcessingPool::threadFunction()
|
||||
{
|
||||
std::unique_lock lock(tasks_mutex);
|
||||
wake_event.wait_for(lock,
|
||||
std::chrono::duration<double>(sleep_seconds
|
||||
+ std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
|
||||
std::chrono::duration<double>(thread_sleep_seconds
|
||||
+ std::uniform_real_distribution<double>(0, thread_sleep_seconds_random_part)(rng)));
|
||||
continue;
|
||||
}
|
||||
|
||||
@ -183,7 +188,7 @@ void BackgroundProcessingPool::threadFunction()
|
||||
{
|
||||
std::unique_lock lock(tasks_mutex);
|
||||
wake_event.wait_for(lock, std::chrono::microseconds(
|
||||
min_time - current_time + std::uniform_int_distribution<uint64_t>(0, sleep_seconds_random_part * 1000000)(rng)));
|
||||
min_time - current_time + std::uniform_int_distribution<uint64_t>(0, thread_sleep_seconds_random_part * 1000000)(rng)));
|
||||
}
|
||||
|
||||
std::shared_lock rlock(task->rwlock);
|
||||
@ -205,16 +210,27 @@ void BackgroundProcessingPool::threadFunction()
|
||||
if (shutdown)
|
||||
break;
|
||||
|
||||
/// If task has done work, it could be executed again immediately.
|
||||
/// If not, add delay before next run.
|
||||
Poco::Timestamp next_time_to_execute = Poco::Timestamp() + (done_work ? 0 : sleep_seconds * 1000000);
|
||||
|
||||
{
|
||||
std::unique_lock lock(tasks_mutex);
|
||||
|
||||
if (task->removed)
|
||||
continue;
|
||||
|
||||
if (done_work)
|
||||
task->count_no_work_done = 0;
|
||||
else
|
||||
++task->count_no_work_done;
|
||||
|
||||
/// If task has done work, it could be executed again immediately.
|
||||
/// If not, add delay before next run.
|
||||
|
||||
Poco::Timestamp next_time_to_execute; /// current time
|
||||
if (!done_work)
|
||||
next_time_to_execute += 1000000 * (std::min(
|
||||
task_sleep_seconds_when_no_work_max,
|
||||
task_sleep_seconds_when_no_work_min * std::pow(task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done))
|
||||
+ std::uniform_real_distribution<double>(0, task_sleep_seconds_when_no_work_random_part)(rng));
|
||||
|
||||
tasks.erase(task->iterator);
|
||||
task->iterator = tasks.emplace(next_time_to_execute, task);
|
||||
}
|
||||
|
@ -55,8 +55,6 @@ protected:
|
||||
using Threads = std::vector<std::thread>;
|
||||
|
||||
const size_t size;
|
||||
static constexpr double sleep_seconds = 10;
|
||||
static constexpr double sleep_seconds_random_part = 1.0;
|
||||
|
||||
Tasks tasks; /// Ordered in priority.
|
||||
std::mutex tasks_mutex;
|
||||
@ -95,6 +93,9 @@ protected:
|
||||
std::atomic<bool> removed {false};
|
||||
|
||||
std::multimap<Poco::Timestamp, std::shared_ptr<BackgroundProcessingPoolTaskInfo>>::iterator iterator;
|
||||
|
||||
/// For exponential backoff.
|
||||
size_t count_no_work_done = 0;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -3,7 +3,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
size_t DiskSpaceMonitor::reserved_bytes;
|
||||
UInt64 DiskSpaceMonitor::reserved_bytes;
|
||||
size_t DiskSpaceMonitor::reservation_count;
|
||||
std::mutex DiskSpaceMonitor::mutex;
|
||||
|
||||
|
@ -68,7 +68,7 @@ public:
|
||||
}
|
||||
|
||||
/// Change amount of reserved space. When new_size is greater than before, availability of free space is not checked.
|
||||
void update(size_t new_size)
|
||||
void update(UInt64 new_size)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
|
||||
DiskSpaceMonitor::reserved_bytes -= size;
|
||||
@ -76,12 +76,12 @@ public:
|
||||
DiskSpaceMonitor::reserved_bytes += size;
|
||||
}
|
||||
|
||||
size_t getSize() const
|
||||
UInt64 getSize() const
|
||||
{
|
||||
return size;
|
||||
}
|
||||
|
||||
Reservation(size_t size_)
|
||||
Reservation(UInt64 size_)
|
||||
: size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
|
||||
@ -90,23 +90,23 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
size_t size;
|
||||
UInt64 size;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
};
|
||||
|
||||
using ReservationPtr = std::unique_ptr<Reservation>;
|
||||
|
||||
static size_t getUnreservedFreeSpace(const std::string & path)
|
||||
static UInt64 getUnreservedFreeSpace(const std::string & path)
|
||||
{
|
||||
struct statvfs fs;
|
||||
|
||||
if (statvfs(path.c_str(), &fs) != 0)
|
||||
throwFromErrno("Could not calculate available disk space (statvfs)", ErrorCodes::CANNOT_STATVFS);
|
||||
|
||||
size_t res = fs.f_bfree * fs.f_bsize;
|
||||
UInt64 res = fs.f_bfree * fs.f_bsize;
|
||||
|
||||
/// Heuristic by Michael Kolupaev: reserve 30 MB more, because statvfs shows few megabytes more space than df.
|
||||
res -= std::min(res, static_cast<size_t>(30 * (1ul << 20)));
|
||||
res -= std::min(res, static_cast<UInt64>(30 * (1ul << 20)));
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
@ -118,22 +118,22 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
static size_t getReservedSpace()
|
||||
static UInt64 getReservedSpace()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return reserved_bytes;
|
||||
}
|
||||
|
||||
static size_t getReservationCount()
|
||||
static UInt64 getReservationCount()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return reservation_count;
|
||||
}
|
||||
|
||||
/// If not enough (approximately) space, throw an exception.
|
||||
static ReservationPtr reserve(const std::string & path, size_t size)
|
||||
static ReservationPtr reserve(const std::string & path, UInt64 size)
|
||||
{
|
||||
size_t free_bytes = getUnreservedFreeSpace(path);
|
||||
UInt64 free_bytes = getUnreservedFreeSpace(path);
|
||||
if (free_bytes < size)
|
||||
throw Exception("Not enough free disk space to reserve: " + formatReadableSizeWithBinarySuffix(free_bytes) + " available, "
|
||||
+ formatReadableSizeWithBinarySuffix(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
|
||||
@ -141,8 +141,8 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
static size_t reserved_bytes;
|
||||
static size_t reservation_count;
|
||||
static UInt64 reserved_bytes;
|
||||
static UInt64 reservation_count;
|
||||
static std::mutex mutex;
|
||||
};
|
||||
|
||||
|
@ -413,6 +413,25 @@ void MergeTreeData::initPartitionKey()
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!encountered_date_column)
|
||||
{
|
||||
for (size_t i = 0; i < minmax_idx_column_types.size(); ++i)
|
||||
{
|
||||
if (typeid_cast<const DataTypeDateTime *>(minmax_idx_column_types[i].get()))
|
||||
{
|
||||
if (!encountered_date_column)
|
||||
{
|
||||
minmax_idx_time_column_pos = i;
|
||||
encountered_date_column = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// There is more than one DateTime column in partition key and we don't know which one to choose.
|
||||
minmax_idx_time_column_pos = -1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -1451,7 +1470,7 @@ void MergeTreeData::AlterDataPartTransaction::commit()
|
||||
file.remove();
|
||||
}
|
||||
|
||||
mutable_part.bytes_on_disk = MergeTreeData::DataPart::calculateTotalSizeOnDisk(path);
|
||||
mutable_part.bytes_on_disk = new_checksums.getTotalSizeOnDisk();
|
||||
|
||||
/// TODO: we can skip resetting caches when the column is added.
|
||||
data_part->storage.context.dropCaches();
|
||||
|
@ -578,6 +578,7 @@ public:
|
||||
Names minmax_idx_columns;
|
||||
DataTypes minmax_idx_column_types;
|
||||
Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column.
|
||||
Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column.
|
||||
|
||||
/// Names of columns for primary key + secondary sorting columns.
|
||||
Names sorting_key_columns;
|
||||
|
@ -127,7 +127,7 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, co
|
||||
}
|
||||
|
||||
|
||||
size_t MergeTreeDataMergerMutator::getMaxSourcePartsSize()
|
||||
UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSize()
|
||||
{
|
||||
size_t total_threads_in_pool = pool.getNumberOfThreads();
|
||||
size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed);
|
||||
@ -136,14 +136,14 @@ size_t MergeTreeDataMergerMutator::getMaxSourcePartsSize()
|
||||
}
|
||||
|
||||
|
||||
size_t MergeTreeDataMergerMutator::getMaxSourcePartsSize(size_t pool_size, size_t pool_used)
|
||||
UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSize(size_t pool_size, size_t pool_used)
|
||||
{
|
||||
if (pool_used > pool_size)
|
||||
throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
size_t free_entries = pool_size - pool_used;
|
||||
|
||||
size_t max_size = 0;
|
||||
UInt64 max_size = 0;
|
||||
if (free_entries >= data.settings.number_of_free_entries_in_pool_to_lower_max_size_of_merge)
|
||||
max_size = data.settings.max_bytes_to_merge_at_max_space_in_pool;
|
||||
else
|
||||
@ -152,7 +152,7 @@ size_t MergeTreeDataMergerMutator::getMaxSourcePartsSize(size_t pool_size, size_
|
||||
data.settings.max_bytes_to_merge_at_max_space_in_pool,
|
||||
static_cast<double>(free_entries) / data.settings.number_of_free_entries_in_pool_to_lower_max_size_of_merge);
|
||||
|
||||
return std::min(max_size, static_cast<size_t>(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_SELECT));
|
||||
return std::min(max_size, static_cast<UInt64>(DiskSpaceMonitor::getUnreservedFreeSpace(data.full_path) / DISK_USAGE_COEFFICIENT_TO_SELECT));
|
||||
}
|
||||
|
||||
|
||||
@ -245,7 +245,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
|
||||
|
||||
bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition(
|
||||
FuturePart & future_part,
|
||||
size_t & available_disk_space,
|
||||
UInt64 & available_disk_space,
|
||||
const AllowedMergingPredicate & can_merge,
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
@ -266,7 +266,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition(
|
||||
auto it = parts.begin();
|
||||
auto prev_it = it;
|
||||
|
||||
size_t sum_bytes = 0;
|
||||
UInt64 sum_bytes = 0;
|
||||
while (it != parts.end())
|
||||
{
|
||||
/// For the case of one part, we check that it can be merged "with itself".
|
||||
|
@ -71,7 +71,7 @@ public:
|
||||
*/
|
||||
bool selectAllPartsToMergeWithinPartition(
|
||||
FuturePart & future_part,
|
||||
size_t & available_disk_space,
|
||||
UInt64 & available_disk_space,
|
||||
const AllowedMergingPredicate & can_merge,
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
|
@ -248,7 +248,6 @@ String MergeTreeDataPart::getNewName(const MergeTreePartInfo & new_part_info) co
|
||||
return new_part_info.getPartName();
|
||||
}
|
||||
|
||||
|
||||
DayNum MergeTreeDataPart::getMinDate() const
|
||||
{
|
||||
if (storage.minmax_idx_date_column_pos != -1 && minmax_idx.initialized)
|
||||
@ -266,6 +265,22 @@ DayNum MergeTreeDataPart::getMaxDate() const
|
||||
return DayNum();
|
||||
}
|
||||
|
||||
time_t MergeTreeDataPart::getMinTime() const
|
||||
{
|
||||
if (storage.minmax_idx_time_column_pos != -1 && minmax_idx.initialized)
|
||||
return minmax_idx.parallelogram[storage.minmax_idx_time_column_pos].left.get<UInt64>();
|
||||
else
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
time_t MergeTreeDataPart::getMaxTime() const
|
||||
{
|
||||
if (storage.minmax_idx_time_column_pos != -1 && minmax_idx.initialized)
|
||||
return minmax_idx.parallelogram[storage.minmax_idx_time_column_pos].right.get<UInt64>();
|
||||
else
|
||||
return 0;
|
||||
}
|
||||
|
||||
MergeTreeDataPart::~MergeTreeDataPart()
|
||||
{
|
||||
@ -479,8 +494,6 @@ void MergeTreeDataPart::loadIndex()
|
||||
|
||||
index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end()));
|
||||
}
|
||||
|
||||
bytes_on_disk = calculateTotalSizeOnDisk(getFullPath());
|
||||
}
|
||||
|
||||
void MergeTreeDataPart::loadPartitionAndMinMaxIndex()
|
||||
@ -514,16 +527,25 @@ void MergeTreeDataPart::loadPartitionAndMinMaxIndex()
|
||||
void MergeTreeDataPart::loadChecksums(bool require)
|
||||
{
|
||||
String path = getFullPath() + "checksums.txt";
|
||||
if (!Poco::File(path).exists())
|
||||
Poco::File checksums_file(path);
|
||||
if (checksums_file.exists())
|
||||
{
|
||||
ReadBufferFromFile file = openForReading(path);
|
||||
if (checksums.read(file))
|
||||
{
|
||||
assertEOF(file);
|
||||
bytes_on_disk = checksums.getTotalSizeOnDisk();
|
||||
}
|
||||
else
|
||||
bytes_on_disk = calculateTotalSizeOnDisk(getFullPath());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (require)
|
||||
throw Exception("No checksums.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
return;
|
||||
bytes_on_disk = calculateTotalSizeOnDisk(getFullPath());
|
||||
}
|
||||
ReadBufferFromFile file = openForReading(path);
|
||||
if (checksums.read(file))
|
||||
assertEOF(file);
|
||||
}
|
||||
|
||||
void MergeTreeDataPart::loadRowsCount()
|
||||
|
@ -75,6 +75,10 @@ struct MergeTreeDataPart
|
||||
DayNum getMinDate() const;
|
||||
DayNum getMaxDate() const;
|
||||
|
||||
/// otherwise, if the partition key includes dateTime column (also a common case), these functions will return min and max values for this column.
|
||||
time_t getMinTime() const;
|
||||
time_t getMaxTime() const;
|
||||
|
||||
bool isEmpty() const { return rows_count == 0; }
|
||||
|
||||
const MergeTreeData & storage;
|
||||
@ -90,6 +94,7 @@ struct MergeTreeDataPart
|
||||
size_t marks_count = 0;
|
||||
std::atomic<UInt64> bytes_on_disk {0}; /// 0 - if not counted;
|
||||
/// Is used from several threads without locks (it is changed with ALTER).
|
||||
/// May not contain size of checksums.txt and columns.txt
|
||||
time_t modification_time = 0;
|
||||
/// When the part is removed from the working set. Changes once.
|
||||
mutable std::atomic<time_t> remove_time { std::numeric_limits<time_t>::max() };
|
||||
|
@ -87,6 +87,14 @@ void MergeTreeDataPartChecksums::checkSizes(const String & path) const
|
||||
}
|
||||
}
|
||||
|
||||
UInt64 MergeTreeDataPartChecksums::getTotalSizeOnDisk() const
|
||||
{
|
||||
UInt64 res = 0;
|
||||
for (const auto & it : files)
|
||||
res += it.second.file_size;
|
||||
return res;
|
||||
}
|
||||
|
||||
bool MergeTreeDataPartChecksums::read(ReadBuffer & in, size_t format_version)
|
||||
{
|
||||
switch (format_version)
|
||||
|
@ -84,6 +84,8 @@ struct MergeTreeDataPartChecksums
|
||||
|
||||
String getSerializedString() const;
|
||||
static MergeTreeDataPartChecksums deserializeFrom(const String & s);
|
||||
|
||||
UInt64 getTotalSizeOnDisk() const;
|
||||
};
|
||||
|
||||
|
||||
|
@ -12,7 +12,7 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
void MergeTreeSettings::loadFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config)
|
||||
void MergeTreeSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
if (!config.has(config_elem))
|
||||
return;
|
||||
|
@ -167,7 +167,7 @@ struct MergeTreeSettings
|
||||
#undef DECLARE
|
||||
|
||||
public:
|
||||
void loadFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
|
||||
void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
/// NOTE: will rewrite the AST to add immutable settings.
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
|
@ -378,7 +378,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
|
||||
new_part->columns = *total_column_list;
|
||||
new_part->index.assign(std::make_move_iterator(index_columns.begin()), std::make_move_iterator(index_columns.end()));
|
||||
new_part->checksums = checksums;
|
||||
new_part->bytes_on_disk = MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_part->getFullPath());
|
||||
new_part->bytes_on_disk = checksums.getTotalSizeOnDisk();
|
||||
}
|
||||
|
||||
void MergedBlockOutputStream::init()
|
||||
|
@ -139,10 +139,14 @@ void registerStorageJoin(StorageFactory & factory)
|
||||
{
|
||||
for (const ASTSetQuery::Change & setting : args.storage_def->settings->changes)
|
||||
{
|
||||
if (setting.name == "join_use_nulls") join_use_nulls.set(setting.value);
|
||||
else if (setting.name == "max_rows_in_join") max_rows_in_join.set(setting.value);
|
||||
else if (setting.name == "max_bytes_in_join") max_bytes_in_join.set(setting.value);
|
||||
else if (setting.name == "join_overflow_mode") join_overflow_mode.set(setting.value);
|
||||
if (setting.name == "join_use_nulls")
|
||||
join_use_nulls.set(setting.value);
|
||||
else if (setting.name == "max_rows_in_join")
|
||||
max_rows_in_join.set(setting.value);
|
||||
else if (setting.name == "max_bytes_in_join")
|
||||
max_bytes_in_join.set(setting.value);
|
||||
else if (setting.name == "join_overflow_mode")
|
||||
join_overflow_mode.set(setting.value);
|
||||
else
|
||||
throw Exception(
|
||||
"Unknown setting " + setting.name + " for storage " + args.engine_name,
|
||||
|
@ -424,7 +424,7 @@ bool StorageMergeTree::merge(
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
selected = merger_mutator.selectAllPartsToMergeWithinPartition(future_part, disk_space, can_merge, partition_id, final, out_disable_reason);
|
||||
}
|
||||
|
||||
|
@ -2948,7 +2948,6 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
|
||||
/// (merge_selecting_thread or OPTIMIZE queries) could assign new merges.
|
||||
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
|
||||
|
||||
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
auto zookeeper = getZooKeeper();
|
||||
ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper);
|
||||
|
||||
@ -2967,6 +2966,8 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
|
||||
for (const MergeTreeData::DataPartPtr & part : data_parts)
|
||||
partition_ids.emplace(part->info.partition_id);
|
||||
|
||||
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
|
||||
for (const String & partition_id : partition_ids)
|
||||
{
|
||||
MergeTreeDataMergerMutator::FuturePart future_merged_part;
|
||||
@ -2989,6 +2990,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
|
||||
}
|
||||
else
|
||||
{
|
||||
UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||
selected = merger_mutator.selectAllPartsToMergeWithinPartition(
|
||||
future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);
|
||||
|
@ -1,4 +1,4 @@
|
||||
// autogenerated by dbms/src/Storages/System/StorageSystemContributors.sh
|
||||
// autogenerated by ./StorageSystemContributors.sh
|
||||
const char * auto_contributors[] {
|
||||
"Alberto",
|
||||
"Aleksandra (Ася)",
|
||||
@ -57,6 +57,7 @@ const char * auto_contributors[] {
|
||||
"Constantin S. Pan",
|
||||
"CurtizJ",
|
||||
"Daniel Bershatsky",
|
||||
"Daniel Dao",
|
||||
"Denis Burlaka",
|
||||
"Denis Zhuravlev",
|
||||
"Derek Perkins",
|
||||
@ -64,8 +65,10 @@ const char * auto_contributors[] {
|
||||
"Dmitry Galuza",
|
||||
"Dmitry Luhtionov",
|
||||
"Dmitry Moskowski",
|
||||
"Dmitry Petukhov",
|
||||
"Dmitry S..ky / skype: dvska-at-skype",
|
||||
"Elghazal Ahmed",
|
||||
"Emmanuel Donin de Rosière",
|
||||
"Eugene Klimov",
|
||||
"Eugene Konkov",
|
||||
"Evgeniy Gatov",
|
||||
@ -86,6 +89,7 @@ const char * auto_contributors[] {
|
||||
"Ilya Khomutov",
|
||||
"Ilya Korolev",
|
||||
"Ilya Shipitsin",
|
||||
"Ivan",
|
||||
"Ivan Babrou",
|
||||
"Ivan Blinkov",
|
||||
"Ivan He",
|
||||
@ -97,6 +101,7 @@ const char * auto_contributors[] {
|
||||
"Keiji Yoshida",
|
||||
"Kirill Malev",
|
||||
"Kirill Shvakov",
|
||||
"Koblikov Mihail",
|
||||
"KochetovNicolai",
|
||||
"Konstantin Grabar",
|
||||
"Konstantin Lebedev",
|
||||
@ -202,6 +207,7 @@ const char * auto_contributors[] {
|
||||
"Yurii Vlasenko",
|
||||
"Yury Karpovich",
|
||||
"Yury Stankevich",
|
||||
"abdrakhmanov",
|
||||
"abyss7",
|
||||
"alesapin",
|
||||
"alexey-milovidov",
|
||||
@ -239,9 +245,12 @@ const char * auto_contributors[] {
|
||||
"leozhang",
|
||||
"liuyimin",
|
||||
"lomberts",
|
||||
"mf5137",
|
||||
"mfridental",
|
||||
"morty",
|
||||
"moscas",
|
||||
"ns-vasilev",
|
||||
"ogorbacheva",
|
||||
"orantius",
|
||||
"peshkurov",
|
||||
"proller",
|
||||
|
@ -31,6 +31,8 @@ StorageSystemParts::StorageSystemParts(const std::string & name)
|
||||
{"refcount", std::make_shared<DataTypeUInt32>()},
|
||||
{"min_date", std::make_shared<DataTypeDate>()},
|
||||
{"max_date", std::make_shared<DataTypeDate>()},
|
||||
{"min_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"max_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"partition_id", std::make_shared<DataTypeString>()},
|
||||
{"min_block_number", std::make_shared<DataTypeInt64>()},
|
||||
{"max_block_number", std::make_shared<DataTypeInt64>()},
|
||||
@ -82,6 +84,8 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns, const Stor
|
||||
|
||||
columns[i++]->insert(part->getMinDate());
|
||||
columns[i++]->insert(part->getMaxDate());
|
||||
columns[i++]->insert(part->getMinTime());
|
||||
columns[i++]->insert(part->getMaxTime());
|
||||
columns[i++]->insert(part->info.partition_id);
|
||||
columns[i++]->insert(part->info.min_block);
|
||||
columns[i++]->insert(part->info.max_block);
|
||||
|
@ -1,93 +0,0 @@
|
||||
#include <Storages/getStructureOfRemoteTable.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/getClusterName.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <TableFunctions/TableFunctionShardByHash.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
StoragePtr TableFunctionShardByHash::executeImpl(const ASTPtr & ast_function, const Context & context) const
|
||||
{
|
||||
ASTs & args_func = typeid_cast<ASTFunction &>(*ast_function).children;
|
||||
|
||||
const char * err = "Table function 'shardByHash' requires 4 parameters: "
|
||||
"cluster name, key string to hash, name of remote database, name of remote table.";
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
|
||||
|
||||
if (args.size() != 4)
|
||||
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
String cluster_name;
|
||||
String key;
|
||||
String remote_database;
|
||||
String remote_table;
|
||||
|
||||
auto getStringLiteral = [](const IAST & node, const char * description)
|
||||
{
|
||||
const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(&node);
|
||||
if (!lit)
|
||||
throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (lit->value.getType() != Field::Types::String)
|
||||
throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return safeGet<const String &>(lit->value);
|
||||
};
|
||||
|
||||
cluster_name = getClusterName(*args[0]);
|
||||
key = getStringLiteral(*args[1], "Key to hash");
|
||||
|
||||
args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(args[2], context);
|
||||
args[3] = evaluateConstantExpressionOrIdentifierAsLiteral(args[3], context);
|
||||
|
||||
remote_database = static_cast<const ASTLiteral &>(*args[2]).value.safeGet<String>();
|
||||
remote_table = static_cast<const ASTLiteral &>(*args[3]).value.safeGet<String>();
|
||||
|
||||
/// Similar to other TableFunctions.
|
||||
for (auto & arg : args)
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
|
||||
id->setSpecial();
|
||||
|
||||
auto cluster = context.getCluster(cluster_name);
|
||||
size_t shard_index = sipHash64(key) % cluster->getShardCount();
|
||||
|
||||
std::shared_ptr<Cluster> shard(cluster->getClusterWithSingleShard(shard_index).release());
|
||||
|
||||
auto res = StorageDistributed::createWithOwnCluster(
|
||||
getName(),
|
||||
getStructureOfRemoteTable(*shard, remote_database, remote_table, context),
|
||||
remote_database,
|
||||
remote_table,
|
||||
shard,
|
||||
context);
|
||||
res->startup();
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionShardByHash(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionShardByHash>();
|
||||
}
|
||||
|
||||
}
|
@ -1,23 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* shardByHash(cluster, 'key', db, table) - creates a temporary StorageDistributed,
|
||||
* using the cluster `cluster`, and selecting from it only one shard by hashing the string key.
|
||||
*
|
||||
* Similarly to the `remote` function, to get the table structure, a DESC TABLE request is made to the remote server.
|
||||
*/
|
||||
class TableFunctionShardByHash : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "shardByHash";
|
||||
std::string getName() const override { return name; }
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context) const override;
|
||||
};
|
||||
|
||||
}
|
@ -8,7 +8,6 @@ namespace DB
|
||||
|
||||
void registerTableFunctionMerge(TableFunctionFactory & factory);
|
||||
void registerTableFunctionRemote(TableFunctionFactory & factory);
|
||||
void registerTableFunctionShardByHash(TableFunctionFactory & factory);
|
||||
void registerTableFunctionNumbers(TableFunctionFactory & factory);
|
||||
void registerTableFunctionCatBoostPool(TableFunctionFactory & factory);
|
||||
void registerTableFunctionFile(TableFunctionFactory & factory);
|
||||
@ -35,7 +34,6 @@ void registerTableFunctions()
|
||||
|
||||
registerTableFunctionMerge(factory);
|
||||
registerTableFunctionRemote(factory);
|
||||
registerTableFunctionShardByHash(factory);
|
||||
registerTableFunctionNumbers(factory);
|
||||
registerTableFunctionCatBoostPool(factory);
|
||||
registerTableFunctionFile(factory);
|
||||
|
@ -177,6 +177,18 @@ def test_postgres_odbc_hached_dictionary_with_schema(started_cluster):
|
||||
assert node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))") == "hello\n"
|
||||
assert node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))") == "world\n"
|
||||
|
||||
def test_postgres_odbc_hached_dictionary_no_tty_pipe_overflow(started_cluster):
|
||||
conn = get_postgres_conn()
|
||||
cursor = conn.cursor()
|
||||
cursor.execute("insert into clickhouse.test_table values(3, 'xxx')")
|
||||
for i in xrange(100):
|
||||
try:
|
||||
node1.query("system reload dictionary postgres_odbc_hashed", timeout=5)
|
||||
except Exception as ex:
|
||||
assert False, "Exception occured -- odbc-bridge hangs: " + str(ex)
|
||||
|
||||
assert node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(3))") == "xxx\n"
|
||||
|
||||
def test_bridge_dies_with_parent(started_cluster):
|
||||
node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))")
|
||||
def get_pid(cmd):
|
||||
|
26
dbms/tests/performance/array/array_join.xml
Normal file
26
dbms/tests/performance/array/array_join.xml
Normal file
@ -0,0 +1,26 @@
|
||||
<test>
|
||||
<name>array_join</name>
|
||||
<type>once</type>
|
||||
|
||||
<stop_conditions>
|
||||
<any_of>
|
||||
<average_speed_not_changing_for_ms>10000</average_speed_not_changing_for_ms>
|
||||
<total_time_ms>1000</total_time_ms>
|
||||
</any_of>
|
||||
</stop_conditions>
|
||||
|
||||
<metrics>
|
||||
<max_rows_per_second />
|
||||
</metrics>
|
||||
|
||||
<main_metric>
|
||||
<max_rows_per_second />
|
||||
</main_metric>
|
||||
|
||||
<query>SELECT count() FROM (SELECT [number] a, [number * 2] b FROM system.numbers) AS t ARRAY JOIN a, b WHERE NOT ignore(a + b)</query>
|
||||
<query>SELECT count() FROM (SELECT [number] a, [number * 2] b FROM system.numbers) AS t LEFT ARRAY JOIN a, b WHERE NOT ignore(a + b)</query>
|
||||
<query>SELECT count() FROM (SELECT [number] a, [number * 2] b FROM system.numbers) AS t ARRAY JOIN a, b WHERE NOT ignore(a + b) SETTINGS enable_unaligned_array_join = 1</query>
|
||||
<query>SELECT count() FROM (SELECT [number] a, [number * 2] b FROM system.numbers) AS t LEFT ARRAY JOIN a, b WHERE NOT ignore(a + b) SETTINGS enable_unaligned_array_join = 1</query>
|
||||
<query>SELECT count() FROM (SELECT [number] a, [number * 2, number] b FROM system.numbers) AS t ARRAY JOIN a, b WHERE NOT ignore(a + b) SETTINGS enable_unaligned_array_join = 1</query>
|
||||
<query>SELECT count() FROM (SELECT [number] a, [number * 2, number] b FROM system.numbers) AS t LEFT ARRAY JOIN a, b WHERE NOT ignore(a + b) SETTINGS enable_unaligned_array_join = 1</query>
|
||||
</test>
|
@ -0,0 +1,24 @@
|
||||
1 [0] [0] 0 0
|
||||
2 [] [0,1] 0 0
|
||||
2 [] [0,1] 0 1
|
||||
3 [0] [] 0 0
|
||||
4 [] [0] 0 0
|
||||
5 [0] [0,1] 0 0
|
||||
5 [0] [0,1] 0 1
|
||||
7 [0] [0] 0 0
|
||||
8 [] [0,1] 0 0
|
||||
8 [] [0,1] 0 1
|
||||
9 [0] [] 0 0
|
||||
0 [] [] 0 0
|
||||
1 [0] [0] 0 0
|
||||
2 [] [0,1] 0 0
|
||||
2 [] [0,1] 0 1
|
||||
3 [0] [] 0 0
|
||||
4 [] [0] 0 0
|
||||
5 [0] [0,1] 0 0
|
||||
5 [0] [0,1] 0 1
|
||||
6 [] [] 0 0
|
||||
7 [0] [0] 0 0
|
||||
8 [] [0,1] 0 0
|
||||
8 [] [0,1] 0 1
|
||||
9 [0] [] 0 0
|
@ -0,0 +1,2 @@
|
||||
SELECT number, arr1, arr2, x, y FROM (SELECT number, range(number % 2) AS arr1, range(number % 3) arr2 FROM system.numbers LIMIT 10) ARRAY JOIN arr1 AS x, arr2 AS y SETTINGS enable_unaligned_array_join = 1;
|
||||
SELECT number, arr1, arr2, x, y FROM (SELECT number, range(number % 2) AS arr1, range(number % 3) arr2 FROM system.numbers LIMIT 10) LEFT ARRAY JOIN arr1 AS x, arr2 AS y SETTINGS enable_unaligned_array_join = 1;
|
@ -0,0 +1,24 @@
|
||||
1981-09-30 23:00:00
|
||||
1981-09-30 23:05:00
|
||||
1981-09-30 23:10:00
|
||||
1981-09-30 23:15:00
|
||||
1981-09-30 23:20:00
|
||||
1981-09-30 23:25:00
|
||||
1981-09-30 23:30:00
|
||||
1981-09-30 23:35:00
|
||||
1981-09-30 23:40:00
|
||||
1981-09-30 23:45:00
|
||||
1981-09-30 23:50:00
|
||||
1981-09-30 23:55:00
|
||||
2018-09-21 23:00:00
|
||||
2018-09-21 23:05:00
|
||||
2018-09-21 23:10:00
|
||||
2018-09-21 23:15:00
|
||||
2018-09-21 23:20:00
|
||||
2018-09-21 23:25:00
|
||||
2018-09-21 23:30:00
|
||||
2018-09-21 23:35:00
|
||||
2018-09-21 23:40:00
|
||||
2018-09-21 23:45:00
|
||||
2018-09-21 23:50:00
|
||||
2018-09-21 23:55:00
|
@ -0,0 +1,3 @@
|
||||
-- concat with empty string to defeat injectiveness of toString assumption.
|
||||
SELECT concat('', toString(toDateTime('1981-09-29 00:00:00', 'Europe/Moscow') + INTERVAL number * 300 SECOND)) AS k FROM numbers(10000) GROUP BY k HAVING count() > 1 ORDER BY k;
|
||||
SELECT concat('', toString(toDateTime('2018-09-19 00:00:00', 'Asia/Tehran') + INTERVAL number * 300 SECOND)) AS k FROM numbers(1000) GROUP BY k HAVING count() > 1 ORDER BY k;
|
@ -0,0 +1,8 @@
|
||||
"2","0000-00-00 00:00:00","0000-00-00 00:00:00"
|
||||
"4","0000-00-00 00:00:00","0000-00-00 00:00:00"
|
||||
"2016-01-01 00:00:00","2016-01-01 00:00:00","2016-01-01 02:00:00"
|
||||
"2016-01-02 00:00:00","2016-01-02 01:00:00","2016-01-02 23:00:00"
|
||||
"2016-01-03 00:00:00","2016-01-03 04:00:00","2016-01-03 04:00:00"
|
||||
"('name1',16801)",1451606400,1451613600
|
||||
"('name1',16803)",1451793600,1451793600
|
||||
"('name2',16802)",1451696400,1451775600
|
@ -0,0 +1,83 @@
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.datetime_table;
|
||||
|
||||
-- Create a table with DateTime column, but not used in partition key
|
||||
CREATE TABLE test.datetime_table
|
||||
(
|
||||
t DateTime,
|
||||
name String,
|
||||
value UInt32
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY (t, name)
|
||||
PARTITION BY value;
|
||||
|
||||
INSERT INTO test.datetime_table VALUES (toDateTime('2016-01-01 00:00:00'),'name1',2);
|
||||
INSERT INTO test.datetime_table VALUES (toDateTime('2016-01-02 00:00:00'),'name2',2);
|
||||
INSERT INTO test.datetime_table VALUES (toDateTime('2016-01-03 00:00:00'),'name1',4);
|
||||
|
||||
-- min_time and max_time are not filled
|
||||
|
||||
SELECT partition, MIN(min_time) as min_time, MAX(max_time) as max_time
|
||||
FROM system.parts
|
||||
WHERE database = 'test' and table = 'datetime_table' AND active = 1
|
||||
GROUP BY partition
|
||||
ORDER BY partition ASC
|
||||
FORMAT CSV;
|
||||
|
||||
DROP TABLE IF EXISTS test.datetime_table;
|
||||
|
||||
-- Create a table with DateTime column, this time used in partition key
|
||||
CREATE TABLE test.datetime_table
|
||||
(
|
||||
t DateTime,
|
||||
name String,
|
||||
value UInt32
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY (t, name)
|
||||
PARTITION BY toStartOfDay(t);
|
||||
|
||||
INSERT INTO test.datetime_table VALUES (toDateTime('2016-01-01 00:00:00'),'name1',2);
|
||||
INSERT INTO test.datetime_table VALUES (toDateTime('2016-01-01 02:00:00'),'name1',3);
|
||||
INSERT INTO test.datetime_table VALUES (toDateTime('2016-01-02 01:00:00'),'name2',2);
|
||||
INSERT INTO test.datetime_table VALUES (toDateTime('2016-01-02 23:00:00'),'name2',5);
|
||||
INSERT INTO test.datetime_table VALUES (toDateTime('2016-01-03 04:00:00'),'name1',4);
|
||||
|
||||
-- min_time and max_time are now filled
|
||||
|
||||
SELECT partition, MIN(min_time) as min_time, MAX(max_time) as max_time
|
||||
FROM system.parts
|
||||
WHERE database = 'test' and table = 'datetime_table' AND active = 1
|
||||
GROUP BY partition
|
||||
ORDER BY partition ASC
|
||||
FORMAT CSV;
|
||||
|
||||
DROP TABLE IF EXISTS test.datetime_table;
|
||||
|
||||
-- Create a table with DateTime column, this time used in partition key, but not at the first level
|
||||
CREATE TABLE test.datetime_table
|
||||
(
|
||||
t DateTime,
|
||||
name String,
|
||||
value UInt32
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY (t, name)
|
||||
PARTITION BY (name, toUInt32(toUnixTimestamp(t)/(60*60*24)) );
|
||||
|
||||
-- We are using a daily aggregation that is independant of the timezone, add data also
|
||||
|
||||
INSERT INTO test.datetime_table VALUES (1451606400,'name1',2);
|
||||
INSERT INTO test.datetime_table VALUES (1451613600,'name1',3);
|
||||
INSERT INTO test.datetime_table VALUES (1451696400,'name2',2);
|
||||
INSERT INTO test.datetime_table VALUES (1451775600,'name2',5);
|
||||
INSERT INTO test.datetime_table VALUES (1451793600,'name1',4);
|
||||
|
||||
-- min_time and max_time are now filled
|
||||
|
||||
SELECT partition, toUnixTimestamp(MIN(min_time)) as min_unix_time, toUnixTimestamp(MAX(max_time)) as max_unix_time
|
||||
FROM system.parts
|
||||
WHERE database = 'test' and table = 'datetime_table' AND active = 1
|
||||
GROUP BY partition
|
||||
ORDER BY partition ASC
|
||||
FORMAT CSV;
|
||||
|
||||
DROP TABLE IF EXISTS test.datetime_table;
|
Binary file not shown.
@ -0,0 +1,2 @@
|
||||
SELECT 1 AS x, [2, 3] AS y, 'Hello' AS z, NULL AS a FORMAT ODBCDriver2;
|
||||
SELECT number % 10 AS k, count() AS c FROM numbers(100) GROUP BY k WITH TOTALS FORMAT ODBCDriver2;
|
@ -0,0 +1,12 @@
|
||||
a \N 1
|
||||
a b 1
|
||||
a \N 2
|
||||
a b 1
|
||||
a \N 1
|
||||
a b 1
|
||||
a \N 2
|
||||
|
||||
\N \N 2
|
||||
a b 1
|
||||
|
||||
\N \N 1
|
15
dbms/tests/queries/0_stateless/00804_rollup_with_having.sql
Normal file
15
dbms/tests/queries/0_stateless/00804_rollup_with_having.sql
Normal file
@ -0,0 +1,15 @@
|
||||
DROP TABLE IF EXISTS test.rollup_having;
|
||||
CREATE TABLE test.rollup_having (
|
||||
a Nullable(String),
|
||||
b Nullable(String)
|
||||
) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test.rollup_having VALUES (NULL, NULL);
|
||||
INSERT INTO test.rollup_having VALUES ('a', NULL);
|
||||
INSERT INTO test.rollup_having VALUES ('a', 'b');
|
||||
|
||||
SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP HAVING a IS NOT NULL;
|
||||
SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP HAVING a IS NOT NULL and b IS NOT NULL;
|
||||
|
||||
SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL;
|
||||
SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL and b IS NOT NULL;
|
4
debian/changelog
vendored
4
debian/changelog
vendored
@ -1,5 +1,5 @@
|
||||
clickhouse (18.14.17) unstable; urgency=low
|
||||
clickhouse (18.16.0) unstable; urgency=low
|
||||
|
||||
* Modified source code
|
||||
|
||||
-- <root@yandex-team.ru> Thu, 29 Nov 2018 17:05:34 +0300
|
||||
-- <root@yandex-team.ru> Fri, 14 Dec 2018 20:26:45 +0300
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=18.14.17
|
||||
ARG version=18.16.0
|
||||
|
||||
RUN apt-get update \
|
||||
&& apt-get install --yes --no-install-recommends \
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=18.14.17
|
||||
ARG version=18.16.0
|
||||
ARG gosu_ver=1.10
|
||||
|
||||
RUN apt-get update \
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=18.14.17
|
||||
ARG version=18.16.0
|
||||
|
||||
RUN apt-get update && \
|
||||
apt-get install -y apt-transport-https dirmngr && \
|
||||
|
@ -50,7 +50,7 @@ SELECT
|
||||
|
||||
## Working with data types
|
||||
|
||||
When creating an array on the fly, ClickHouse automatically defines the argument type as the narrowest data type that can store all the listed arguments. If there are any [NULL](../query_language/syntax.md#null-literal) or [Nullable](nullable.md#data_type-nullable) type arguments, the type of array elements is [Nullable](nullable.md#data_type-nullable).
|
||||
When creating an array on the fly, ClickHouse automatically defines the argument type as the narrowest data type that can store all the listed arguments. If there are any [NULL](../query_language/syntax.md#null-literal) or [Nullable](nullable.md#data_type-nullable) type arguments, the type of array elements is [Nullable](nullable.md).
|
||||
|
||||
If ClickHouse couldn't determine the data type, it will generate an exception. For instance, this will happen when trying to create an array with strings and numbers simultaneously (`SELECT array(1, 'a')`).
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
<a name="data_type-decimal"></a>
|
||||
|
||||
# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S)
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
<a name="data_type-enum"></a>
|
||||
|
||||
# Enum8, Enum16
|
||||
|
||||
@ -77,9 +76,9 @@ SELECT toTypeName(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)'))
|
||||
|
||||
Each of the values is assigned a number in the range `-128 ... 127` for `Enum8` or in the range `-32768 ... 32767` for `Enum16`. All the strings and numbers must be different. An empty string is allowed. If this type is specified (in a table definition), numbers can be in an arbitrary order. However, the order does not matter.
|
||||
|
||||
Neither the string nor the numeric value in an `Enum` can be [NULL](../query_language/syntax.md#null-literal).
|
||||
Neither the string nor the numeric value in an `Enum` can be [NULL](../query_language/syntax.md).
|
||||
|
||||
An `Enum` can be contained in [Nullable](nullable.md#data_type-nullable) type. So if you create a table using the query
|
||||
An `Enum` can be contained in [Nullable](nullable.md) type. So if you create a table using the query
|
||||
|
||||
```
|
||||
CREATE TABLE t_enum_nullable
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user