diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionDebug.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionDebug.h new file mode 100644 index 00000000000..11c8e0531ba --- /dev/null +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionDebug.h @@ -0,0 +1,90 @@ +#include + + +namespace DB +{ + + +/** Сделано в целях отладки. Подлежит удалению. + */ + +struct AggregateFunctionDebugData +{ + UInt32 value; + + AggregateFunctionDebugData() + { + value = 0xAAAAAAAA; + + if (rand() % 1000 == 0) + throw Exception("Test1"); + } + + ~AggregateFunctionDebugData() + { + try + { + if (value == 0xDEADDEAD) + throw Exception("Double free"); + + if (value != 0xAAAAAAAA) + throw Exception("Corruption"); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + std::terminate(); + } + + value = 0xDEADDEAD; + } +}; + +class AggregateFunctionDebug final : public IUnaryAggregateFunction +{ +public: + String getName() const { return "debug"; } + + DataTypePtr getReturnType() const + { + return new DataTypeUInt32; + } + + void setArgument(const DataTypePtr & argument) + { + } + + void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const + { + if (rand() % 1000 == 0) + throw Exception("Test2"); + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const + { + if (rand() % 1000 == 0) + throw Exception("Test3"); + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const + { + if (rand() % 1000 == 0) + throw Exception("Test4"); + } + + void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const + { + if (rand() % 1000 == 0) + throw Exception("Test5"); + } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const + { + if (rand() % 1000 == 0) + throw Exception("Test6"); + + static_cast(to).getData().push_back(123); + } +}; + +} diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h index 96bdd1791a8..f4783d380bf 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h @@ -147,10 +147,10 @@ struct __attribute__((__packed__)) SingleValueDataString static constexpr Int32 AUTOMATIC_STORAGE_SIZE = 64; static constexpr Int32 MAX_SMALL_STRING_SIZE = AUTOMATIC_STORAGE_SIZE - sizeof(size); - union + union __attribute__((__aligned__(1))) { char small_data[MAX_SMALL_STRING_SIZE]; /// Включая завершающий ноль. - char * large_data; + char * __attribute__((__aligned__(1))) large_data; }; ~SingleValueDataString() diff --git a/dbms/include/DB/Client/Connection.h b/dbms/include/DB/Client/Connection.h index 2d968532101..f1994178fdc 100644 --- a/dbms/include/DB/Client/Connection.h +++ b/dbms/include/DB/Client/Connection.h @@ -55,13 +55,15 @@ public: Protocol::Compression::Enum compression_ = Protocol::Compression::Enable, Poco::Timespan connect_timeout_ = Poco::Timespan(DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, 0), Poco::Timespan receive_timeout_ = Poco::Timespan(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, 0), - Poco::Timespan send_timeout_ = Poco::Timespan(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0)) + Poco::Timespan send_timeout_ = Poco::Timespan(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0), + Poco::Timespan ping_timeout_ = Poco::Timespan(DBMS_DEFAULT_PING_TIMEOUT_SEC, 0)) : host(host_), port(port_), default_database(default_database_), user(user_), password(password_), client_name(client_name_), compression(compression_), data_type_factory(data_type_factory_), connect_timeout(connect_timeout_), receive_timeout(receive_timeout_), send_timeout(send_timeout_), + ping_timeout(ping_timeout_), log_wrapper(host, port) { /// Соединеняемся не сразу, а при первой необходимости. @@ -178,6 +180,7 @@ private: Poco::Timespan connect_timeout; Poco::Timespan receive_timeout; Poco::Timespan send_timeout; + Poco::Timespan ping_timeout; /// Откуда читать результат выполнения запроса. SharedPtr maybe_compressed_in; diff --git a/dbms/include/DB/Client/ConnectionPoolWithFailover.h b/dbms/include/DB/Client/ConnectionPoolWithFailover.h index 4057cd51675..994e4fbee31 100644 --- a/dbms/include/DB/Client/ConnectionPoolWithFailover.h +++ b/dbms/include/DB/Client/ConnectionPoolWithFailover.h @@ -91,6 +91,8 @@ private: void applyLoadBalancing(Settings * settings) { + Poco::ScopedLock lock(mutex); + LoadBalancing load_balancing = default_load_balancing; if (settings) load_balancing = settings->load_balancing; diff --git a/dbms/include/DB/Columns/ColumnFixedString.h b/dbms/include/DB/Columns/ColumnFixedString.h index d362adee5ef..c2ecba68768 100644 --- a/dbms/include/DB/Columns/ColumnFixedString.h +++ b/dbms/include/DB/Columns/ColumnFixedString.h @@ -18,10 +18,13 @@ public: typedef PODArray Chars_t; private: + /// Байты строк, уложенные подряд. Строки хранятся без завершающего нулевого байта. + /** NOTE Требуется, чтобы смещение и тип chars в объекте был таким же, как у data в ColumnVector. + * Это используется в функции packFixed (AggregationCommon.h) + */ + Chars_t chars; /// Размер строк. const size_t n; - /// Байты строк, уложенные подряд. Строки хранятся без завершающего нулевого байта. - Chars_t chars; public: /** Создать пустой столбец строк фиксированной длины n */ diff --git a/dbms/include/DB/Common/HashTable/HashTable.h b/dbms/include/DB/Common/HashTable/HashTable.h index ab725fc1e6d..dba467c83e8 100644 --- a/dbms/include/DB/Common/HashTable/HashTable.h +++ b/dbms/include/DB/Common/HashTable/HashTable.h @@ -583,7 +583,21 @@ protected: if (unlikely(grower.overflow(m_size))) { - resize(); + try + { + resize(); + } + catch (...) + { + /** Если этого не делать, то будут проблемы. + * Ведь останется ключ, но неинициализированное mapped-значение, + * у которого, возможно, даже нельзя вызвать деструктор. + */ + --m_size; + buf[place_value].setZero(); + throw; + } + it = find(x, hash_value); } } diff --git a/dbms/include/DB/Common/PODArray.h b/dbms/include/DB/Common/PODArray.h index 0b4e0550a30..222dfa39ec5 100644 --- a/dbms/include/DB/Common/PODArray.h +++ b/dbms/include/DB/Common/PODArray.h @@ -52,7 +52,7 @@ private: char * c_end; char * c_end_of_storage; - bool use_libc_realloc; + bool use_libc_realloc = false; T * t_start() { return reinterpret_cast(c_start); } T * t_end() { return reinterpret_cast(c_end); } @@ -131,18 +131,22 @@ private: if (use_libc_realloc) { - c_start = reinterpret_cast(::realloc(c_start, bytes_to_alloc)); + auto new_c_start = reinterpret_cast(::realloc(c_start, bytes_to_alloc)); - if (nullptr == c_start) + if (nullptr == new_c_start) throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY); + + c_start = new_c_start; } else { - c_start = reinterpret_cast(malloc(bytes_to_alloc)); + auto new_c_start = reinterpret_cast(malloc(bytes_to_alloc)); - if (nullptr == c_start) + if (nullptr == new_c_start) throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY); + c_start = new_c_start; + memcpy(c_start, old_c_start, std::min(bytes_to_alloc, static_cast(end_diff))); Allocator::deallocate(old_c_start, old_c_end_of_storage - old_c_start); } @@ -171,10 +175,10 @@ public: }; - PODArray() : use_libc_realloc(false) { alloc(0); } - PODArray(size_t n) : use_libc_realloc(false) { alloc(n); c_end += byte_size(n); } - PODArray(size_t n, const T & x) : use_libc_realloc(false) { alloc(n); assign(n, x); } - PODArray(const_iterator from_begin, const_iterator from_end) : use_libc_realloc(false) { alloc(from_end - from_begin); insert(from_begin, from_end); } + PODArray() { alloc(0); } + PODArray(size_t n) { alloc(n); c_end += byte_size(n); } + PODArray(size_t n, const T & x) { alloc(n); assign(n, x); } + PODArray(const_iterator from_begin, const_iterator from_end) { alloc(from_end - from_begin); insert(from_begin, from_end); } ~PODArray() { dealloc(); } PODArray(PODArray && other) { *this = std::move(other); } diff --git a/dbms/include/DB/Core/Defines.h b/dbms/include/DB/Core/Defines.h index 0a147a2287e..ced6e2ddfcf 100644 --- a/dbms/include/DB/Core/Defines.h +++ b/dbms/include/DB/Core/Defines.h @@ -11,6 +11,7 @@ #define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS 50 #define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300 #define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300 +#define DBMS_DEFAULT_PING_TIMEOUT_SEC 5 #define DBMS_DEFAULT_POLL_INTERVAL 10 /// Насколько секунд можно максимально задерживать вставку в таблицу типа MergeTree, если в ней много недомердженных кусков. @@ -46,7 +47,9 @@ #define DEFAULT_INTERACTIVE_DELAY 100000 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 #define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3 -#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD 300 /// каждый период уменьшаем счетчик ошибок в 2 раза +/// каждый период уменьшаем счетчик ошибок в 2 раза +/// слишком маленький период может приводить, что ошибки исчезают сразу после создания. +#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD (2*DBMS_DEFAULT_SEND_TIMEOUT_SEC) #define DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS 5000 /// Максимальное время ожидания в очереди запросов. #define DBMS_DEFAULT_BACKGROUND_POOL_SIZE 6 @@ -68,5 +71,8 @@ #define DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS 100 +/// Граница, на которых должны быть выровнены блоки для асинхронных файловых операций. +#define DEFAULT_AIO_FILE_BLOCK_SIZE 512 + #define ALWAYS_INLINE __attribute__((__always_inline__)) #define NO_INLINE __attribute__((__noinline__)) diff --git a/dbms/include/DB/Core/ErrorCodes.h b/dbms/include/DB/Core/ErrorCodes.h index 02a0c3113a0..8f7e087456b 100644 --- a/dbms/include/DB/Core/ErrorCodes.h +++ b/dbms/include/DB/Core/ErrorCodes.h @@ -277,6 +277,14 @@ namespace ErrorCodes STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS, CPUID_ERROR, INFINITE_LOOP, + CANNOT_COMPRESS, + CANNOT_DECOMPRESS, + AIO_SUBMIT_ERROR, + AIO_COMPLETION_ERROR, + AIO_READ_ERROR, + AIO_WRITE_ERROR, + AIO_UNALIGNED_SIZE_ERROR, + INDEX_NOT_USED, POCO_EXCEPTION = 1000, STD_EXCEPTION, diff --git a/dbms/include/DB/DataStreams/AggregatingBlockInputStream.h b/dbms/include/DB/DataStreams/AggregatingBlockInputStream.h index c9e5191a5fe..0187c1281de 100644 --- a/dbms/include/DB/DataStreams/AggregatingBlockInputStream.h +++ b/dbms/include/DB/DataStreams/AggregatingBlockInputStream.h @@ -18,16 +18,6 @@ using Poco::SharedPtr; class AggregatingBlockInputStream : public IProfilingBlockInputStream { public: - AggregatingBlockInputStream(BlockInputStreamPtr input_, const ColumnNumbers & keys_, AggregateDescriptions & aggregates_, - bool overflow_row_, bool final_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, - Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_) - : aggregator(keys_, aggregates_, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_, - compiler_, min_count_to_compile_, group_by_two_level_threshold_), - final(final_) - { - children.push_back(input_); - } - /** keys берутся из GROUP BY части запроса * Агрегатные функции ищутся везде в выражении. * Столбцы, соответствующие keys и аргументам агрегатных функций, уже должны быть вычислены. diff --git a/dbms/include/DB/DataStreams/CollapsingFinalBlockInputStream.h b/dbms/include/DB/DataStreams/CollapsingFinalBlockInputStream.h index cc89c946059..11334cd4d3b 100644 --- a/dbms/include/DB/DataStreams/CollapsingFinalBlockInputStream.h +++ b/dbms/include/DB/DataStreams/CollapsingFinalBlockInputStream.h @@ -15,10 +15,8 @@ class CollapsingFinalBlockInputStream : public IProfilingBlockInputStream { public: CollapsingFinalBlockInputStream(BlockInputStreams inputs_, const SortDescription & description_, - const String & sign_column_) - : description(description_), sign_column(sign_column_), - log(&Logger::get("CollapsingFinalBlockInputStream")), - first(true), count_positive(0), count_negative(0), count_incorrect_data(0), blocks_fetched(0), blocks_output(0) + const String & sign_column_name_) + : description(description_), sign_column_name(sign_column_name_) { children.insert(children.end(), inputs_.begin(), inputs_.end()); } @@ -40,7 +38,7 @@ public: for (size_t i = 0; i < description.size(); ++i) res << ", " << description[i].getID(); - res << ", sign_column, " << sign_column << ")"; + res << ", sign_column, " << sign_column_name << ")"; return res.str(); } @@ -55,10 +53,10 @@ private: { MergingBlock(Block block_, size_t stream_index_, - SortDescription desc_, + const SortDescription & desc, String sign_column_name, BlockPlainPtrs * output_blocks) - : block(block_), stream_index(stream_index_), desc(desc_), refcount(0), output_blocks(output_blocks) + : block(block_), stream_index(stream_index_), output_blocks(output_blocks) { sort_columns.resize(desc.size()); for (size_t i = 0; i < desc.size(); ++i) @@ -86,8 +84,6 @@ private: /// Строки с одинаковым ключом будут упорядочены по возрастанию stream_index. size_t stream_index; - - SortDescription desc; size_t rows; /// Какие строки нужно оставить. Заполняется при слиянии потоков. @@ -98,7 +94,7 @@ private: const ColumnInt8 * sign_column; /// Когда достигает нуля, блок можно выдавать в ответ. - int refcount; + int refcount = 0; /// Куда положить блок, когда он готов попасть в ответ. BlockPlainPtrs * output_blocks; @@ -181,17 +177,17 @@ private: Cursor() {} explicit Cursor(MergingBlockPtr block_, size_t pos_ = 0) : block(block_), pos(pos_) {} - bool operator<(const Cursor & rhs) const + bool operator< (const Cursor & rhs) const { for (size_t i = 0; i < block->sort_columns.size(); ++i) { - int direction = block->desc[i].direction; - int res = direction * block->sort_columns[i]->compareAt(pos, rhs.pos, *(rhs.block->sort_columns[i]), direction); + int res = block->sort_columns[i]->compareAt(pos, rhs.pos, *(rhs.block->sort_columns[i]), 1); if (res > 0) return true; if (res < 0) return false; } + return block->stream_index > rhs.block->stream_index; } @@ -203,7 +199,7 @@ private: for (size_t i = 0; i < block->sort_columns.size(); ++i) { - int res = block->desc[i].direction * block->sort_columns[i]->compareAt(pos, rhs.pos, *(rhs.block->sort_columns[i]), 1); + int res = block->sort_columns[i]->compareAt(pos, rhs.pos, *(rhs.block->sort_columns[i]), 1); if (res != 0) return false; } @@ -235,12 +231,12 @@ private: typedef std::priority_queue Queue; - SortDescription description; - String sign_column; + const SortDescription description; + String sign_column_name; - Logger * log; + Logger * log = &Logger::get("CollapsingFinalBlockInputStream"); - bool first; + bool first = true; BlockPlainPtrs output_blocks; @@ -249,15 +245,15 @@ private: Cursor previous; /// Текущий первичный ключ. Cursor last_positive; /// Последняя положительная строка для текущего первичного ключа. - size_t count_positive; /// Количество положительных строк для текущего первичного ключа. - size_t count_negative; /// Количество отрицательных строк для текущего первичного ключа. - bool last_is_positive; /// true, если последняя строка для текущего первичного ключа положительная. + size_t count_positive = 0; /// Количество положительных строк для текущего первичного ключа. + size_t count_negative = 0; /// Количество отрицательных строк для текущего первичного ключа. + bool last_is_positive = false; /// true, если последняя строка для текущего первичного ключа положительная. - size_t count_incorrect_data; /// Чтобы не писать в лог слишком много сообщений об ошибке. + size_t count_incorrect_data = 0; /// Чтобы не писать в лог слишком много сообщений об ошибке. /// Посчитаем, сколько блоков получили на вход и отдали на выход. - size_t blocks_fetched; - size_t blocks_output; + size_t blocks_fetched = 0; + size_t blocks_output = 0; void fetchNextBlock(size_t input_index); void commitCurrent(); diff --git a/dbms/include/DB/DataStreams/IProfilingBlockInputStream.h b/dbms/include/DB/DataStreams/IProfilingBlockInputStream.h index ac04ae593d8..88374fb19e4 100644 --- a/dbms/include/DB/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/include/DB/DataStreams/IProfilingBlockInputStream.h @@ -7,9 +7,9 @@ #include #include - #include +#include namespace DB { @@ -96,9 +96,9 @@ public: /** Требуется ли прервать получение данных. */ - bool isCancelled() + bool isCancelled() const { - return is_cancelled; + return is_cancelled.load(std::memory_order_seq_cst); } /** Какие ограничения (и квоты) проверяются. @@ -117,27 +117,19 @@ public: /// Используется подмножество ограничений из Limits. struct LocalLimits { - LimitsMode mode; + LimitsMode mode = LIMITS_CURRENT; - size_t max_rows_to_read; - size_t max_bytes_to_read; - OverflowMode read_overflow_mode; + size_t max_rows_to_read = 0; + size_t max_bytes_to_read = 0; + OverflowMode read_overflow_mode = OverflowMode::THROW; - Poco::Timespan max_execution_time; - OverflowMode timeout_overflow_mode; + Poco::Timespan max_execution_time = 0; + OverflowMode timeout_overflow_mode = OverflowMode::THROW; /// В строчках в секунду. - size_t min_execution_speed; + size_t min_execution_speed = 0; /// Проверять, что скорость не слишком низкая, после прошествия указанного времени. - Poco::Timespan timeout_before_checking_execution_speed; - - LocalLimits() - : mode(LIMITS_CURRENT), - max_rows_to_read(0), max_bytes_to_read(0), read_overflow_mode(OverflowMode::THROW), - max_execution_time(0), timeout_overflow_mode(OverflowMode::THROW), - min_execution_speed(0), timeout_before_checking_execution_speed(0) - { - } + Poco::Timespan timeout_before_checking_execution_speed = 0; }; /** Установить ограничения для проверки на каждый блок. */ @@ -159,7 +151,7 @@ public: protected: BlockStreamProfileInfo info; - volatile bool is_cancelled = false; + std::atomic is_cancelled{false}; ProgressCallback progress_callback; ProcessList::Element * process_list_elem = nullptr; diff --git a/dbms/include/DB/DataStreams/MergingAggregatedBlockInputStream.h b/dbms/include/DB/DataStreams/MergingAggregatedBlockInputStream.h index 1e0a9f71c8d..5330585a835 100644 --- a/dbms/include/DB/DataStreams/MergingAggregatedBlockInputStream.h +++ b/dbms/include/DB/DataStreams/MergingAggregatedBlockInputStream.h @@ -16,14 +16,6 @@ using Poco::SharedPtr; class MergingAggregatedBlockInputStream : public IProfilingBlockInputStream { public: - MergingAggregatedBlockInputStream(BlockInputStreamPtr input_, const ColumnNumbers & keys_, - const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_) - : aggregator(keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0), - final(final_), max_threads(max_threads_) - { - children.push_back(input_); - } - MergingAggregatedBlockInputStream(BlockInputStreamPtr input_, const Names & keys_names_, const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_) : aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0), diff --git a/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h index f813596c63e..8a339a435f2 100644 --- a/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h @@ -19,19 +19,6 @@ using Poco::SharedPtr; class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream { public: - ParallelAggregatingBlockInputStream(BlockInputStreams inputs, const ColumnNumbers & keys_, - AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_, - size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, - Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_) - : aggregator(keys_, aggregates_, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_, - compiler_, min_count_to_compile_, group_by_two_level_threshold_), - final(final_), max_threads(std::min(inputs.size(), max_threads_)), - keys_size(keys_.size()), aggregates_size(aggregates_.size()), - handler(*this), processor(inputs, max_threads, handler) - { - children.insert(children.end(), inputs.begin(), inputs.end()); - } - /** Столбцы из key_names и аргументы агрегатных функций, уже должны быть вычислены. */ ParallelAggregatingBlockInputStream(BlockInputStreams inputs, const Names & key_names, @@ -41,7 +28,7 @@ public: : aggregator(key_names, aggregates, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_, compiler_, min_count_to_compile_, group_by_two_level_threshold_), final(final_), max_threads(std::min(inputs.size(), max_threads_)), - keys_size(key_names.size()), aggregates_size(aggregates.size()), + keys_size(aggregator.getNumberOfKeys()), aggregates_size(aggregator.getNumberOfAggregates()), handler(*this), processor(inputs, max_threads, handler) { children.insert(children.end(), inputs.begin(), inputs.end()); @@ -70,7 +57,8 @@ public: void cancel() override { - if (!__sync_bool_compare_and_swap(&is_cancelled, false, true)) + bool old_val = false; + if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) return; processor.cancel(); @@ -122,6 +110,31 @@ private: Logger * log = &Logger::get("ParallelAggregatingBlockInputStream"); + ManyAggregatedDataVariants many_data; + Exceptions exceptions; + + struct ThreadData + { + size_t src_rows = 0; + size_t src_bytes = 0; + + StringRefs key; + ConstColumnPlainPtrs key_columns; + Aggregator::AggregateColumns aggregate_columns; + Sizes key_sizes; + + ThreadData(size_t keys_size, size_t aggregates_size) + { + key.resize(keys_size); + key_columns.resize(keys_size); + aggregate_columns.resize(aggregates_size); + key_sizes.resize(keys_size); + } + }; + + std::vector threads_data; + + struct Handler { Handler(ParallelAggregatingBlockInputStream & parent_) @@ -154,29 +167,6 @@ private: Handler handler; ParallelInputsProcessor processor; - ManyAggregatedDataVariants many_data; - Exceptions exceptions; - - struct ThreadData - { - size_t src_rows = 0; - size_t src_bytes = 0; - - StringRefs key; - ConstColumnPlainPtrs key_columns; - Aggregator::AggregateColumns aggregate_columns; - Sizes key_sizes; - - ThreadData(size_t keys_size, size_t aggregates_size) - { - key.resize(keys_size); - key_columns.resize(keys_size); - aggregate_columns.resize(aggregates_size); - key_sizes.resize(keys_size); - } - }; - - std::vector threads_data; AggregatedDataVariantsPtr executeAndMerge() { diff --git a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h index d00e1fb4c39..06cd6f2a466 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h @@ -80,7 +80,8 @@ public: void cancel() override { - if (!__sync_bool_compare_and_swap(&is_cancelled, false, true)) + bool old_val = false; + if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) return; if (hasNoQueryInProgress() || hasThrownException()) @@ -147,6 +148,9 @@ protected: while (true) { + if (isCancelled()) + return Block(); + Connection::Packet packet = parallel_replicas->receivePacket(); switch (packet.type) @@ -178,10 +182,6 @@ protected: * и квот (например, на количество строчек для чтения). */ progressImpl(packet.progress); - - if (isQueryInProgress() && isCancelled()) - cancel(); - break; case Protocol::Server::ProfileInfo: @@ -281,7 +281,7 @@ private: void tryCancel(const char * reason) { bool old_val = false; - if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_seq_cst)) + if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) return; LOG_TRACE(log, "(" << parallel_replicas->dumpAddresses() << ") " << reason); diff --git a/dbms/include/DB/DataStreams/UnionBlockInputStream.h b/dbms/include/DB/DataStreams/UnionBlockInputStream.h index 7813ae8f81b..60e163e13f1 100644 --- a/dbms/include/DB/DataStreams/UnionBlockInputStream.h +++ b/dbms/include/DB/DataStreams/UnionBlockInputStream.h @@ -76,7 +76,8 @@ public: */ void cancel() override { - if (!__sync_bool_compare_and_swap(&is_cancelled, false, true)) + bool old_val = false; + if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) return; //std::cerr << "cancelling\n"; @@ -164,7 +165,7 @@ protected: void readSuffix() override { //std::cerr << "readSuffix\n"; - if (!all_read && !is_cancelled) + if (!all_read && !is_cancelled.load(std::memory_order_seq_cst)) throw Exception("readSuffix called before all data is read", ErrorCodes::LOGICAL_ERROR); finalize(); diff --git a/dbms/include/DB/DataStreams/copyData.h b/dbms/include/DB/DataStreams/copyData.h index 0f2014594b3..6d74d23bf17 100644 --- a/dbms/include/DB/DataStreams/copyData.h +++ b/dbms/include/DB/DataStreams/copyData.h @@ -6,6 +6,7 @@ #include #include +#include namespace DB { @@ -13,7 +14,7 @@ namespace DB /** Копирует данные из InputStream в OutputStream * (например, из БД в консоль и т. п.) */ -void copyData(IBlockInputStream & from, IBlockOutputStream & to, volatile bool * is_cancelled = nullptr); +void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic * is_cancelled = nullptr); void copyData(IRowInputStream & from, IRowOutputStream & to); } diff --git a/dbms/include/DB/DataTypes/DataTypeFactory.h b/dbms/include/DB/DataTypes/DataTypeFactory.h index 6ba9d80256b..5924370e1a0 100644 --- a/dbms/include/DB/DataTypes/DataTypeFactory.h +++ b/dbms/include/DB/DataTypes/DataTypeFactory.h @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -12,7 +14,7 @@ namespace DB /** Позволяет создать тип данных по его имени. */ -class DataTypeFactory +class DataTypeFactory : public Singleton { public: DataTypeFactory(); diff --git a/dbms/include/DB/Dictionaries/CacheDictionary.h b/dbms/include/DB/Dictionaries/CacheDictionary.h index dd0b55e856d..939c5fdf6a2 100644 --- a/dbms/include/DB/Dictionaries/CacheDictionary.h +++ b/dbms/include/DB/Dictionaries/CacheDictionary.h @@ -68,11 +68,11 @@ public: getItems(*hierarchical_attribute, ids, out); } -#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \ +#define DECLARE_INDIVIDUAL_GETTER(TYPE) \ TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\ {\ auto & attribute = getAttribute(attribute_name);\ - if (attribute.type != AttributeType::LC_TYPE)\ + if (attribute.type != AttributeUnderlyingType::TYPE)\ throw Exception{\ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ ErrorCodes::TYPE_MISMATCH\ @@ -83,21 +83,21 @@ public: getItems(attribute, ids, out);\ return out.front();\ } - DECLARE_INDIVIDUAL_GETTER(UInt8, uint8) - DECLARE_INDIVIDUAL_GETTER(UInt16, uint16) - DECLARE_INDIVIDUAL_GETTER(UInt32, uint32) - DECLARE_INDIVIDUAL_GETTER(UInt64, uint64) - DECLARE_INDIVIDUAL_GETTER(Int8, int8) - DECLARE_INDIVIDUAL_GETTER(Int16, int16) - DECLARE_INDIVIDUAL_GETTER(Int32, int32) - DECLARE_INDIVIDUAL_GETTER(Int64, int64) - DECLARE_INDIVIDUAL_GETTER(Float32, float32) - DECLARE_INDIVIDUAL_GETTER(Float64, float64) + DECLARE_INDIVIDUAL_GETTER(UInt8) + DECLARE_INDIVIDUAL_GETTER(UInt16) + DECLARE_INDIVIDUAL_GETTER(UInt32) + DECLARE_INDIVIDUAL_GETTER(UInt64) + DECLARE_INDIVIDUAL_GETTER(Int8) + DECLARE_INDIVIDUAL_GETTER(Int16) + DECLARE_INDIVIDUAL_GETTER(Int32) + DECLARE_INDIVIDUAL_GETTER(Int64) + DECLARE_INDIVIDUAL_GETTER(Float32) + DECLARE_INDIVIDUAL_GETTER(Float64) #undef DECLARE_INDIVIDUAL_GETTER String getString(const std::string & attribute_name, const id_t id) const override { auto & attribute = getAttribute(attribute_name); - if (attribute.type != AttributeType::string) + if (attribute.type != AttributeUnderlyingType::String) throw Exception{ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH @@ -110,11 +110,11 @@ public: return String{out.getDataAt(0)}; }; -#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\ +#define DECLARE_MULTIPLE_GETTER(TYPE)\ void get##TYPE(const std::string & attribute_name, const PODArray & ids, PODArray & out) const override\ {\ auto & attribute = getAttribute(attribute_name);\ - if (attribute.type != AttributeType::LC_TYPE)\ + if (attribute.type != AttributeUnderlyingType::TYPE)\ throw Exception{\ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ ErrorCodes::TYPE_MISMATCH\ @@ -122,21 +122,21 @@ public: \ getItems(attribute, ids, out);\ } - DECLARE_MULTIPLE_GETTER(UInt8, uint8) - DECLARE_MULTIPLE_GETTER(UInt16, uint16) - DECLARE_MULTIPLE_GETTER(UInt32, uint32) - DECLARE_MULTIPLE_GETTER(UInt64, uint64) - DECLARE_MULTIPLE_GETTER(Int8, int8) - DECLARE_MULTIPLE_GETTER(Int16, int16) - DECLARE_MULTIPLE_GETTER(Int32, int32) - DECLARE_MULTIPLE_GETTER(Int64, int64) - DECLARE_MULTIPLE_GETTER(Float32, float32) - DECLARE_MULTIPLE_GETTER(Float64, float64) + DECLARE_MULTIPLE_GETTER(UInt8) + DECLARE_MULTIPLE_GETTER(UInt16) + DECLARE_MULTIPLE_GETTER(UInt32) + DECLARE_MULTIPLE_GETTER(UInt64) + DECLARE_MULTIPLE_GETTER(Int8) + DECLARE_MULTIPLE_GETTER(Int16) + DECLARE_MULTIPLE_GETTER(Int32) + DECLARE_MULTIPLE_GETTER(Int64) + DECLARE_MULTIPLE_GETTER(Float32) + DECLARE_MULTIPLE_GETTER(Float64) #undef DECLARE_MULTIPLE_GETTER void getString(const std::string & attribute_name, const PODArray & ids, ColumnString * out) const override { auto & attribute = getAttribute(attribute_name); - if (attribute.type != AttributeType::string) + if (attribute.type != AttributeUnderlyingType::String) throw Exception{ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH @@ -154,7 +154,7 @@ private: struct attribute_t final { - AttributeType type; + AttributeUnderlyingType type; std::tupletype != AttributeType::uint64) + if (hierarchical_attribute->type != AttributeUnderlyingType::UInt64) throw Exception{ "Hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH @@ -196,54 +195,54 @@ private: } } - attribute_t createAttributeWithType(const AttributeType type, const std::string & null_value) + attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) { attribute_t attr{type}; switch (type) { - case AttributeType::uint8: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::UInt8: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::uint16: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::UInt16: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::uint32: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::UInt32: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::uint64: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::UInt64: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::int8: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::Int8: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::int16: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::Int16: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::int32: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::Int32: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::int64: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::Int64: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::float32: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::Float32: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::float64: - std::get(attr.null_values) = DB::parse(null_value); + case AttributeUnderlyingType::Float64: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; - case AttributeType::string: - std::get(attr.null_values) = null_value; + case AttributeUnderlyingType::String: + std::get(attr.null_values) = null_value.get(); std::get>(attr.arrays) = std::make_unique(size); break; } @@ -495,17 +494,17 @@ private: { switch (attribute.type) { - case AttributeType::uint8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::uint16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::uint32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::uint64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::int8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::int16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::int32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::int64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::float32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::float64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeType::string: + case AttributeUnderlyingType::UInt8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::UInt16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::UInt32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::UInt64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::Int8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::Int16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::Int32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::Int64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::Float32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::Float64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::String: { const auto & null_value_ref = std::get(attribute.null_values); auto & string_ref = std::get>(attribute.arrays)[idx]; @@ -525,17 +524,17 @@ private: { switch (attribute.type) { - case AttributeType::uint8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::uint16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::uint32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::uint64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::int8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::int16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::int32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::int64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::float32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::float64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeType::string: + case AttributeUnderlyingType::UInt8: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::UInt16: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::UInt32: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::UInt64: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::Int8: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::Int16: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::Int32: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::Int64: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::Float32: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::Float64: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::String: { const auto & string = value.get(); auto & string_ref = std::get>(attribute.arrays)[idx]; diff --git a/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h b/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h index e268769bcc9..7be319a79cb 100644 --- a/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h +++ b/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h @@ -35,13 +35,13 @@ public: max_connections, host, port, db, user, password, context.getDataTypeFactory(), "ClickHouseDictionarySource") }, - load_all_query{composeLoadAllQuery(sample_block, table)} + load_all_query{composeLoadAllQuery(sample_block, db, table)} {} /// copy-constructor is provided in order to support cloneability ClickHouseDictionarySource(const ClickHouseDictionarySource & other) : host{other.host}, port{other.port}, user{other.user}, password{other.password}, - db{other.db}, table{other.db}, + db{other.db}, table{other.table}, sample_block{other.sample_block}, context(other.context), is_local{other.is_local}, pool{is_local ? nullptr : std::make_unique( @@ -75,7 +75,7 @@ public: DictionarySourcePtr clone() const override { return std::make_unique(*this); } private: - static std::string composeLoadAllQuery(const Block & block, const std::string & table) + static std::string composeLoadAllQuery(const Block & block, const std::string & db, const std::string & table) { std::string query; @@ -94,6 +94,8 @@ private: } writeString(" FROM ", out); + writeProbablyBackQuotedString(db, out); + writeChar('.', out); writeProbablyBackQuotedString(table, out); writeChar(';', out); } @@ -121,6 +123,8 @@ private: const auto & id_column_name = sample_block.getByPosition(0).name; writeString(" FROM ", out); + writeProbablyBackQuotedString(db, out); + writeChar('.', out); writeProbablyBackQuotedString(table, out); writeString(" WHERE ", out); writeProbablyBackQuotedString(id_column_name, out); diff --git a/dbms/include/DB/Dictionaries/DictionarySourceFactory.h b/dbms/include/DB/Dictionaries/DictionarySourceFactory.h index 810f87e5d46..865275901f1 100644 --- a/dbms/include/DB/Dictionaries/DictionarySourceFactory.h +++ b/dbms/include/DB/Dictionaries/DictionarySourceFactory.h @@ -15,7 +15,7 @@ namespace DB namespace { -Block createSampleBlock(const DictionaryStructure & dict_struct, const Context & context) +Block createSampleBlock(const DictionaryStructure & dict_struct) { Block block{ ColumnWithNameAndType{ @@ -26,12 +26,9 @@ Block createSampleBlock(const DictionaryStructure & dict_struct, const Context & }; for (const auto & attribute : dict_struct.attributes) - { - const auto & type = context.getDataTypeFactory().get(attribute.type); block.insert(ColumnWithNameAndType{ - type->createColumn(), type, attribute.name + attribute.type->createColumn(), attribute.type, attribute.name }); - } return block; } @@ -55,7 +52,7 @@ public: ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG }; - auto sample_block = createSampleBlock(dict_struct, context); + auto sample_block = createSampleBlock(dict_struct); const auto & source_type = keys.front(); diff --git a/dbms/include/DB/Dictionaries/DictionaryStructure.h b/dbms/include/DB/Dictionaries/DictionaryStructure.h index 15ed7946e1f..6f46010e0d3 100644 --- a/dbms/include/DB/Dictionaries/DictionaryStructure.h +++ b/dbms/include/DB/Dictionaries/DictionaryStructure.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include #include #include #include @@ -9,35 +11,37 @@ namespace DB { -enum class AttributeType +enum class AttributeUnderlyingType { - uint8, - uint16, - uint32, - uint64, - int8, - int16, - int32, - int64, - float32, - float64, - string + UInt8, + UInt16, + UInt32, + UInt64, + Int8, + Int16, + Int32, + Int64, + Float32, + Float64, + String }; -inline AttributeType getAttributeTypeByName(const std::string & type) +inline AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type) { - static const std::unordered_map dictionary{ - { "UInt8", AttributeType::uint8 }, - { "UInt16", AttributeType::uint16 }, - { "UInt32", AttributeType::uint32 }, - { "UInt64", AttributeType::uint64 }, - { "Int8", AttributeType::int8 }, - { "Int16", AttributeType::int16 }, - { "Int32", AttributeType::int32 }, - { "Int64", AttributeType::int64 }, - { "Float32", AttributeType::float32 }, - { "Float64", AttributeType::float64 }, - { "String", AttributeType::string }, + static const std::unordered_map dictionary{ + { "UInt8", AttributeUnderlyingType::UInt8 }, + { "UInt16", AttributeUnderlyingType::UInt16 }, + { "UInt32", AttributeUnderlyingType::UInt32 }, + { "UInt64", AttributeUnderlyingType::UInt64 }, + { "Int8", AttributeUnderlyingType::Int8 }, + { "Int16", AttributeUnderlyingType::Int16 }, + { "Int32", AttributeUnderlyingType::Int32 }, + { "Int64", AttributeUnderlyingType::Int64 }, + { "Float32", AttributeUnderlyingType::Float32 }, + { "Float64", AttributeUnderlyingType::Float64 }, + { "String", AttributeUnderlyingType::String }, + { "Date", AttributeUnderlyingType::UInt16 }, + { "DateTime", AttributeUnderlyingType::UInt32 }, }; const auto it = dictionary.find(type); @@ -50,21 +54,21 @@ inline AttributeType getAttributeTypeByName(const std::string & type) }; } -inline std::string toString(const AttributeType type) +inline std::string toString(const AttributeUnderlyingType type) { switch (type) { - case AttributeType::uint8: return "UInt8"; - case AttributeType::uint16: return "UInt16"; - case AttributeType::uint32: return "UInt32"; - case AttributeType::uint64: return "UInt64"; - case AttributeType::int8: return "Int8"; - case AttributeType::int16: return "Int16"; - case AttributeType::int32: return "Int32"; - case AttributeType::int64: return "Int64"; - case AttributeType::float32: return "Float32"; - case AttributeType::float64: return "Float64"; - case AttributeType::string: return "String"; + case AttributeUnderlyingType::UInt8: return "UInt8"; + case AttributeUnderlyingType::UInt16: return "UInt16"; + case AttributeUnderlyingType::UInt32: return "UInt32"; + case AttributeUnderlyingType::UInt64: return "UInt64"; + case AttributeUnderlyingType::Int8: return "Int8"; + case AttributeUnderlyingType::Int16: return "Int16"; + case AttributeUnderlyingType::Int32: return "Int32"; + case AttributeUnderlyingType::Int64: return "Int64"; + case AttributeUnderlyingType::Float32: return "Float32"; + case AttributeUnderlyingType::Float64: return "Float64"; + case AttributeUnderlyingType::String: return "String"; } throw Exception{ @@ -91,7 +95,7 @@ struct DictionaryLifetime /** Holds the description of a single dictionary attribute: * - name, used for lookup into dictionary and source; -* - type, used in conjunction with DataTypeFactory and getAttributeTypeByname; +* - type, used in conjunction with DataTypeFactory and getAttributeUnderlyingTypeByname; * - null_value, used as a default value for non-existent entries in the dictionary, * decimal representation for numeric attributes; * - hierarchical, whether this attribute defines a hierarchy; @@ -100,8 +104,9 @@ struct DictionaryLifetime struct DictionaryAttribute { std::string name; - std::string type; - std::string null_value; + AttributeUnderlyingType underlying_type; + DataTypePtr type; + Field null_value; bool hierarchical; bool injective; }; @@ -131,12 +136,20 @@ struct DictionaryStructure continue; const auto prefix = config_prefix + '.' + key + '.'; + const auto name = config.getString(prefix + "name"); - const auto type = config.getString(prefix + "type"); - const auto null_value = config.getString(prefix + "null_value"); + const auto type_string = config.getString(prefix + "type"); + const auto type = DataTypeFactory::instance().get(type_string); + const auto underlying_type = getAttributeUnderlyingType(type_string); + + const auto null_value_string = config.getString(prefix + "null_value"); + Field null_value; + ReadBufferFromString null_value_buffer{null_value_string}; + type->deserializeText(null_value, null_value_buffer); + const auto hierarchical = config.getBool(prefix + "hierarchical", false); const auto injective = config.getBool(prefix + "injective", false); - if (name.empty() || type.empty()) + if (name.empty()) throw Exception{ "Properties 'name' and 'type' of an attribute cannot be empty", ErrorCodes::BAD_ARGUMENTS @@ -151,7 +164,7 @@ struct DictionaryStructure has_hierarchy = has_hierarchy || hierarchical; attributes.emplace_back(DictionaryAttribute{ - name, type, null_value, hierarchical, injective + name, underlying_type, type, null_value, hierarchical, injective }); } diff --git a/dbms/include/DB/Dictionaries/FlatDictionary.h b/dbms/include/DB/Dictionaries/FlatDictionary.h index 72b55217569..756255d1bf6 100644 --- a/dbms/include/DB/Dictionaries/FlatDictionary.h +++ b/dbms/include/DB/Dictionaries/FlatDictionary.h @@ -57,11 +57,11 @@ public: getItems(*hierarchical_attribute, ids, out); } -#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \ +#define DECLARE_INDIVIDUAL_GETTER(TYPE) \ TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\ {\ const auto & attribute = getAttribute(attribute_name);\ - if (attribute.type != AttributeType::LC_TYPE)\ + if (attribute.type != AttributeUnderlyingType::TYPE)\ throw Exception{\ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ ErrorCodes::TYPE_MISMATCH\ @@ -71,21 +71,21 @@ public: \ return id < array.size() ? array[id] : std::get(attribute.null_values);\ } - DECLARE_INDIVIDUAL_GETTER(UInt8, uint8) - DECLARE_INDIVIDUAL_GETTER(UInt16, uint16) - DECLARE_INDIVIDUAL_GETTER(UInt32, uint32) - DECLARE_INDIVIDUAL_GETTER(UInt64, uint64) - DECLARE_INDIVIDUAL_GETTER(Int8, int8) - DECLARE_INDIVIDUAL_GETTER(Int16, int16) - DECLARE_INDIVIDUAL_GETTER(Int32, int32) - DECLARE_INDIVIDUAL_GETTER(Int64, int64) - DECLARE_INDIVIDUAL_GETTER(Float32, float32) - DECLARE_INDIVIDUAL_GETTER(Float64, float64) + DECLARE_INDIVIDUAL_GETTER(UInt8) + DECLARE_INDIVIDUAL_GETTER(UInt16) + DECLARE_INDIVIDUAL_GETTER(UInt32) + DECLARE_INDIVIDUAL_GETTER(UInt64) + DECLARE_INDIVIDUAL_GETTER(Int8) + DECLARE_INDIVIDUAL_GETTER(Int16) + DECLARE_INDIVIDUAL_GETTER(Int32) + DECLARE_INDIVIDUAL_GETTER(Int64) + DECLARE_INDIVIDUAL_GETTER(Float32) + DECLARE_INDIVIDUAL_GETTER(Float64) #undef DECLARE_INDIVIDUAL_GETTER String getString(const std::string & attribute_name, const id_t id) const override { const auto & attribute = getAttribute(attribute_name); - if (attribute.type != AttributeType::string) + if (attribute.type != AttributeUnderlyingType::String) throw Exception{ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH @@ -96,11 +96,11 @@ public: return id < array.size() ? String{array[id]} : std::get(attribute.null_values); } -#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\ +#define DECLARE_MULTIPLE_GETTER(TYPE)\ void get##TYPE(const std::string & attribute_name, const PODArray & ids, PODArray & out) const override\ {\ const auto & attribute = getAttribute(attribute_name);\ - if (attribute.type != AttributeType::LC_TYPE)\ + if (attribute.type != AttributeUnderlyingType::TYPE)\ throw Exception{\ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ ErrorCodes::TYPE_MISMATCH\ @@ -108,21 +108,21 @@ public: \ getItems(attribute, ids, out);\ } - DECLARE_MULTIPLE_GETTER(UInt8, uint8) - DECLARE_MULTIPLE_GETTER(UInt16, uint16) - DECLARE_MULTIPLE_GETTER(UInt32, uint32) - DECLARE_MULTIPLE_GETTER(UInt64, uint64) - DECLARE_MULTIPLE_GETTER(Int8, int8) - DECLARE_MULTIPLE_GETTER(Int16, int16) - DECLARE_MULTIPLE_GETTER(Int32, int32) - DECLARE_MULTIPLE_GETTER(Int64, int64) - DECLARE_MULTIPLE_GETTER(Float32, float32) - DECLARE_MULTIPLE_GETTER(Float64, float64) + DECLARE_MULTIPLE_GETTER(UInt8) + DECLARE_MULTIPLE_GETTER(UInt16) + DECLARE_MULTIPLE_GETTER(UInt32) + DECLARE_MULTIPLE_GETTER(UInt64) + DECLARE_MULTIPLE_GETTER(Int8) + DECLARE_MULTIPLE_GETTER(Int16) + DECLARE_MULTIPLE_GETTER(Int32) + DECLARE_MULTIPLE_GETTER(Int64) + DECLARE_MULTIPLE_GETTER(Float32) + DECLARE_MULTIPLE_GETTER(Float64) #undef DECLARE_MULTIPLE_GETTER void getString(const std::string & attribute_name, const PODArray & ids, ColumnString * out) const override { const auto & attribute = getAttribute(attribute_name); - if (attribute.type != AttributeType::string) + if (attribute.type != AttributeUnderlyingType::String) throw Exception{ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH @@ -142,7 +142,7 @@ public: private: struct attribute_t final { - AttributeType type; + AttributeUnderlyingType type; std::tupletype != AttributeType::uint64) + if (hierarchical_attribute->type != AttributeUnderlyingType::UInt64) throw Exception{ "Hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH @@ -208,38 +207,39 @@ private: } template - void createAttributeImpl(attribute_t & attribute, const std::string & null_value) + void createAttributeImpl(attribute_t & attribute, const Field & null_value) { - const auto & null_value_ref = std::get(attribute.null_values) = DB::parse(null_value); + const auto & null_value_ref = std::get(attribute.null_values) = + null_value.get::Type>(); std::get>>(attribute.arrays) = std::make_unique>(initial_array_size, null_value_ref); } - attribute_t createAttributeWithType(const AttributeType type, const std::string & null_value) + attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) { attribute_t attr{type}; switch (type) { - case AttributeType::uint8: createAttributeImpl(attr, null_value); break; - case AttributeType::uint16: createAttributeImpl(attr, null_value); break; - case AttributeType::uint32: createAttributeImpl(attr, null_value); break; - case AttributeType::uint64: createAttributeImpl(attr, null_value); break; - case AttributeType::int8: createAttributeImpl(attr, null_value); break; - case AttributeType::int16: createAttributeImpl(attr, null_value); break; - case AttributeType::int32: createAttributeImpl(attr, null_value); break; - case AttributeType::int64: createAttributeImpl(attr, null_value); break; - case AttributeType::float32: createAttributeImpl(attr, null_value); break; - case AttributeType::float64: createAttributeImpl(attr, null_value); break; - case AttributeType::string: + case AttributeUnderlyingType::UInt8: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::UInt16: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::UInt32: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::UInt64: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Int8: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Int16: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Int32: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Int64: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Float32: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Float64: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::String: { - const auto & null_value_ref = std::get(attr.null_values) = DB::parse(null_value); + const auto & null_value_ref = std::get(attr.null_values) = null_value.get(); std::get>>(attr.arrays) = std::make_unique>(initial_array_size, null_value_ref); attr.string_arena = std::make_unique(); break; } - }; + } return attr; } @@ -276,17 +276,17 @@ private: switch (attribute.type) { - case AttributeType::uint8: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::uint16: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::uint32: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::uint64: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::int8: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::int16: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::int32: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::int64: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::float32: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::float64: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::string: + case AttributeUnderlyingType::UInt8: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::UInt16: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::UInt32: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::UInt64: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Int8: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Int16: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Int32: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Int64: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Float32: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Float64: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::String: { auto & array = *std::get>>(attribute.arrays); if (id >= array.size()) diff --git a/dbms/include/DB/Dictionaries/HashedDictionary.h b/dbms/include/DB/Dictionaries/HashedDictionary.h index 4a942a59fd2..ef664046540 100644 --- a/dbms/include/DB/Dictionaries/HashedDictionary.h +++ b/dbms/include/DB/Dictionaries/HashedDictionary.h @@ -56,11 +56,11 @@ public: getItems(*hierarchical_attribute, ids, out); } -#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \ +#define DECLARE_INDIVIDUAL_GETTER(TYPE) \ TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\ {\ const auto & attribute = getAttribute(attribute_name);\ - if (attribute.type != AttributeType::LC_TYPE)\ + if (attribute.type != AttributeUnderlyingType::TYPE)\ throw Exception{\ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ ErrorCodes::TYPE_MISMATCH\ @@ -71,21 +71,21 @@ public: \ return it != map.end() ? TYPE{it->second} : std::get(attribute.null_values);\ } - DECLARE_INDIVIDUAL_GETTER(UInt8, uint8) - DECLARE_INDIVIDUAL_GETTER(UInt16, uint16) - DECLARE_INDIVIDUAL_GETTER(UInt32, uint32) - DECLARE_INDIVIDUAL_GETTER(UInt64, uint64) - DECLARE_INDIVIDUAL_GETTER(Int8, int8) - DECLARE_INDIVIDUAL_GETTER(Int16, int16) - DECLARE_INDIVIDUAL_GETTER(Int32, int32) - DECLARE_INDIVIDUAL_GETTER(Int64, int64) - DECLARE_INDIVIDUAL_GETTER(Float32, float32) - DECLARE_INDIVIDUAL_GETTER(Float64, float64) + DECLARE_INDIVIDUAL_GETTER(UInt8) + DECLARE_INDIVIDUAL_GETTER(UInt16) + DECLARE_INDIVIDUAL_GETTER(UInt32) + DECLARE_INDIVIDUAL_GETTER(UInt64) + DECLARE_INDIVIDUAL_GETTER(Int8) + DECLARE_INDIVIDUAL_GETTER(Int16) + DECLARE_INDIVIDUAL_GETTER(Int32) + DECLARE_INDIVIDUAL_GETTER(Int64) + DECLARE_INDIVIDUAL_GETTER(Float32) + DECLARE_INDIVIDUAL_GETTER(Float64) #undef DECLARE_INDIVIDUAL_GETTER String getString(const std::string & attribute_name, const id_t id) const override { const auto & attribute = getAttribute(attribute_name); - if (attribute.type != AttributeType::string) + if (attribute.type != AttributeUnderlyingType::String) throw Exception{ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH @@ -97,11 +97,11 @@ public: return it != map.end() ? String{it->second} : std::get(attribute.null_values); } -#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\ +#define DECLARE_MULTIPLE_GETTER(TYPE)\ void get##TYPE(const std::string & attribute_name, const PODArray & ids, PODArray & out) const override\ {\ const auto & attribute = getAttribute(attribute_name);\ - if (attribute.type != AttributeType::LC_TYPE)\ + if (attribute.type != AttributeUnderlyingType::TYPE)\ throw Exception{\ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ ErrorCodes::TYPE_MISMATCH\ @@ -109,21 +109,21 @@ public: \ getItems(attribute, ids, out);\ } - DECLARE_MULTIPLE_GETTER(UInt8, uint8) - DECLARE_MULTIPLE_GETTER(UInt16, uint16) - DECLARE_MULTIPLE_GETTER(UInt32, uint32) - DECLARE_MULTIPLE_GETTER(UInt64, uint64) - DECLARE_MULTIPLE_GETTER(Int8, int8) - DECLARE_MULTIPLE_GETTER(Int16, int16) - DECLARE_MULTIPLE_GETTER(Int32, int32) - DECLARE_MULTIPLE_GETTER(Int64, int64) - DECLARE_MULTIPLE_GETTER(Float32, float32) - DECLARE_MULTIPLE_GETTER(Float64, float64) + DECLARE_MULTIPLE_GETTER(UInt8) + DECLARE_MULTIPLE_GETTER(UInt16) + DECLARE_MULTIPLE_GETTER(UInt32) + DECLARE_MULTIPLE_GETTER(UInt64) + DECLARE_MULTIPLE_GETTER(Int8) + DECLARE_MULTIPLE_GETTER(Int16) + DECLARE_MULTIPLE_GETTER(Int32) + DECLARE_MULTIPLE_GETTER(Int64) + DECLARE_MULTIPLE_GETTER(Float32) + DECLARE_MULTIPLE_GETTER(Float64) #undef DECLARE_MULTIPLE_GETTER void getString(const std::string & attribute_name, const PODArray & ids, ColumnString * out) const override { const auto & attribute = getAttribute(attribute_name); - if (attribute.type != AttributeType::string) + if (attribute.type != AttributeUnderlyingType::String) throw Exception{ "Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH @@ -143,7 +143,7 @@ public: private: struct attribute_t final { - AttributeType type; + AttributeUnderlyingType type; std::tupletype != AttributeType::uint64) + if (hierarchical_attribute->type != AttributeUnderlyingType::UInt64) throw Exception{ "Hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH @@ -209,31 +208,31 @@ private: } template - void createAttributeImpl(attribute_t & attribute, const std::string & null_value) + void createAttributeImpl(attribute_t & attribute, const Field & null_value) { - std::get(attribute.null_values) = DB::parse(null_value); + std::get(attribute.null_values) = null_value.get::Type>(); std::get>>(attribute.maps) = std::make_unique>(); } - attribute_t createAttributeWithType(const AttributeType type, const std::string & null_value) + attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) { attribute_t attr{type}; switch (type) { - case AttributeType::uint8: createAttributeImpl(attr, null_value); break; - case AttributeType::uint16: createAttributeImpl(attr, null_value); break; - case AttributeType::uint32: createAttributeImpl(attr, null_value); break; - case AttributeType::uint64: createAttributeImpl(attr, null_value); break; - case AttributeType::int8: createAttributeImpl(attr, null_value); break; - case AttributeType::int16: createAttributeImpl(attr, null_value); break; - case AttributeType::int32: createAttributeImpl(attr, null_value); break; - case AttributeType::int64: createAttributeImpl(attr, null_value); break; - case AttributeType::float32: createAttributeImpl(attr, null_value); break; - case AttributeType::float64: createAttributeImpl(attr, null_value); break; - case AttributeType::string: + case AttributeUnderlyingType::UInt8: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::UInt16: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::UInt32: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::UInt64: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Int8: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Int16: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Int32: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Int64: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Float32: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Float64: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::String: { - const auto & null_value_ref = std::get(attr.null_values) = DB::parse(null_value); + const auto & null_value_ref = std::get(attr.null_values) = null_value.get(); std::get>>(attr.maps) = std::make_unique>(); attr.string_arena = std::make_unique(); @@ -268,17 +267,17 @@ private: { switch (attribute.type) { - case AttributeType::uint8: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::uint16: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::uint32: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::uint64: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::int8: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::int16: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::int32: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::int64: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::float32: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::float64: setAttributeValueImpl(attribute, id, value.get()); break; - case AttributeType::string: + case AttributeUnderlyingType::UInt8: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::UInt16: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::UInt32: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::UInt64: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Int8: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Int16: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Int32: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Int64: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Float32: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::Float64: setAttributeValueImpl(attribute, id, value.get()); break; + case AttributeUnderlyingType::String: { auto & map = *std::get>>(attribute.maps); const auto & string = value.get(); diff --git a/dbms/include/DB/Dictionaries/MySQLBlockInputStream.h b/dbms/include/DB/Dictionaries/MySQLBlockInputStream.h index 26942ad9b28..d8a14e3c988 100644 --- a/dbms/include/DB/Dictionaries/MySQLBlockInputStream.h +++ b/dbms/include/DB/Dictionaries/MySQLBlockInputStream.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -15,9 +17,30 @@ namespace DB /// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining class MySQLBlockInputStream final : public IProfilingBlockInputStream { + enum struct value_type_t + { + UInt8, + UInt16, + UInt32, + UInt64, + Int8, + Int16, + Int32, + Int64, + Float32, + Float64, + String, + Date, + DateTime + }; + public: - MySQLBlockInputStream(mysqlxx::Query query, const Block & sample_block, const std::size_t max_block_size) - : query{std::move(query)}, result{query.use()}, sample_block{sample_block}, max_block_size{max_block_size} + MySQLBlockInputStream(const mysqlxx::PoolWithFailover::Entry & entry, + const std::string & query_str, + const Block & sample_block, + const std::size_t max_block_size) + : entry{entry}, query{this->entry->query(query_str)}, result{query.use()}, + sample_block{sample_block}, max_block_size{max_block_size} { types.reserve(sample_block.columns()); @@ -25,27 +48,31 @@ public: { const auto type = sample_block.getByPosition(idx).type.get(); if (typeid_cast(type)) - types.push_back(AttributeType::uint8); + types.push_back(value_type_t::UInt8); else if (typeid_cast(type)) - types.push_back(AttributeType::uint16); + types.push_back(value_type_t::UInt16); else if (typeid_cast(type)) - types.push_back(AttributeType::uint32); + types.push_back(value_type_t::UInt32); else if (typeid_cast(type)) - types.push_back(AttributeType::uint64); + types.push_back(value_type_t::UInt64); else if (typeid_cast(type)) - types.push_back(AttributeType::int8); + types.push_back(value_type_t::Int8); else if (typeid_cast(type)) - types.push_back(AttributeType::int16); + types.push_back(value_type_t::Int16); else if (typeid_cast(type)) - types.push_back(AttributeType::int32); + types.push_back(value_type_t::Int32); else if (typeid_cast(type)) - types.push_back(AttributeType::int64); + types.push_back(value_type_t::Int64); else if (typeid_cast(type)) - types.push_back(AttributeType::float32); + types.push_back(value_type_t::Float32); else if (typeid_cast(type)) - types.push_back(AttributeType::float64); + types.push_back(value_type_t::Float64); else if (typeid_cast(type)) - types.push_back(AttributeType::string); + types.push_back(value_type_t::String); + else if (typeid_cast(type)) + types.push_back(value_type_t::Date); + else if (typeid_cast(type)) + types.push_back(value_type_t::DateTime); else throw Exception{ "Unsupported type " + type->getName(), @@ -76,6 +103,7 @@ private: std::size_t rows = 0; while (auto row = result.fetch()) { + /// @todo cache pointers returned by the calls to getByPosition for (const auto idx : ext::range(0, row.size())) insertValue(block.getByPosition(idx).column, row[idx], types[idx]); @@ -87,29 +115,32 @@ private: return rows == 0 ? Block{} : block; }; - static void insertValue(ColumnPtr & column, const mysqlxx::Value & value, const AttributeType type) + static void insertValue(ColumnPtr & column, const mysqlxx::Value & value, const value_type_t type) { switch (type) { - case AttributeType::uint8: column->insert(static_cast(value)); break; - case AttributeType::uint16: column->insert(static_cast(value)); break; - case AttributeType::uint32: column->insert(static_cast(value)); break; - case AttributeType::uint64: column->insert(static_cast(value)); break; - case AttributeType::int8: column->insert(static_cast(value)); break; - case AttributeType::int16: column->insert(static_cast(value)); break; - case AttributeType::int32: column->insert(static_cast(value)); break; - case AttributeType::int64: column->insert(static_cast(value)); break; - case AttributeType::float32: column->insert(static_cast(value)); break; - case AttributeType::float64: column->insert(static_cast(value)); break; - case AttributeType::string: column->insert(value.getString()); break; - } + case value_type_t::UInt8: column->insert(static_cast(value)); break; + case value_type_t::UInt16: column->insert(static_cast(value)); break; + case value_type_t::UInt32: column->insert(static_cast(value)); break; + case value_type_t::UInt64: column->insert(static_cast(value)); break; + case value_type_t::Int8: column->insert(static_cast(value)); break; + case value_type_t::Int16: column->insert(static_cast(value)); break; + case value_type_t::Int32: column->insert(static_cast(value)); break; + case value_type_t::Int64: column->insert(static_cast(value)); break; + case value_type_t::Float32: column->insert(static_cast(value)); break; + case value_type_t::Float64: column->insert(static_cast(value)); break; + case value_type_t::String: column->insert(value.getString()); break; + case value_type_t::Date: column->insert(static_cast(UInt16{value.getDate().getDayNum()})); break; + case value_type_t::DateTime: column->insert(static_cast(time_t{value.getDateTime()})); break; + }; } + mysqlxx::PoolWithFailover::Entry entry; mysqlxx::Query query; mysqlxx::UseQueryResult result; Block sample_block; const std::size_t max_block_size; - std::vector types; + std::vector types; }; } diff --git a/dbms/include/DB/Dictionaries/MySQLDictionarySource.h b/dbms/include/DB/Dictionaries/MySQLDictionarySource.h index 952a4bad9fa..3818ddd6605 100644 --- a/dbms/include/DB/Dictionaries/MySQLDictionarySource.h +++ b/dbms/include/DB/Dictionaries/MySQLDictionarySource.h @@ -36,7 +36,7 @@ public: BlockInputStreamPtr loadAll() override { last_modification = getLastModification(); - return new MySQLBlockInputStream{pool.Get()->query(load_all_query), sample_block, max_block_size}; + return new MySQLBlockInputStream{pool.Get(), load_all_query, sample_block, max_block_size}; } BlockInputStreamPtr loadIds(const std::vector ids) override @@ -44,7 +44,7 @@ public: last_modification = getLastModification(); const auto query = composeLoadIdsQuery(ids); - return new MySQLBlockInputStream{pool.Get()->query(query), sample_block, max_block_size}; + return new MySQLBlockInputStream{pool.Get(), query, sample_block, max_block_size}; } bool isModified() const override { return getLastModification() > last_modification; } @@ -56,16 +56,24 @@ private: mysqlxx::DateTime getLastModification() const { const auto Update_time_idx = 12; + mysqlxx::DateTime update_time{std::time(nullptr)}; try { auto connection = pool.Get(); auto query = connection->query("SHOW TABLE STATUS LIKE '%" + strconvert::escaped_for_like(table) + "%';"); auto result = query.use(); - auto row = result.fetch(); - const auto & update_time = row[Update_time_idx]; - if (!update_time.isNull()) - return update_time.getDateTime(); + + if (auto row = result.fetch()) + { + const auto & update_time_value = row[Update_time_idx]; + + if (!update_time_value.isNull()) + update_time = update_time_value.getDateTime(); + + /// fetch remaining rows to avoid "commands out of sync" error + while (auto row = result.fetch()); + } } catch (...) { @@ -73,7 +81,7 @@ private: } /// we suppose failure to get modification time is not an error, therefore return current time - return mysqlxx::DateTime{std::time(nullptr)}; + return update_time; } static std::string composeLoadAllQuery(const Block & block, const std::string & table) diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index dd1618eb5f1..4c6b41b3545 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -134,6 +134,65 @@ struct ConvertImpl }; +/** Отдельный случай для преобразования UInt32 или UInt64 в Date. + * Если число меньше 65536, то оно понимается, как DayNum, а если больше - как unix timestamp. + * Немного нелогично, что мы, по сути, помещаем две разные функции в одну. + * Но зато это позволяет поддержать распространённый случай, + * когда пользователь пишет toDate(UInt32), ожидая, что это - перевод unix timestamp в дату + * (иначе такое использование было бы распространённой ошибкой). + */ +template +struct ConvertImplUInt32Or64ToDate +{ + typedef typename FromDataType::FieldType FromFieldType; + typedef DataTypeDate::FieldType ToFieldType; + + template + static To convert(const From & from, const DateLUT & date_lut) + { + return from < 0xFFFF + ? from + : date_lut.toDayNum(from); + } + + static void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + DateLUT & date_lut = DateLUT::instance(); + + if (const ColumnVector * col_from + = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + const typename ColumnVector::Container_t & vec_from = col_from->getData(); + typename ColumnVector::Container_t & vec_to = col_to->getData(); + size_t size = vec_from.size(); + vec_to.resize(size); + + for (size_t i = 0; i < size; ++i) + vec_to[i] = convert(vec_from[i], date_lut); + } + else if (const ColumnConst * col_from + = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + { + block.getByPosition(result).column = new ColumnConst(col_from->size(), + convert(col_from->getData(), date_lut)); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +template +struct ConvertImpl : ConvertImplUInt32Or64ToDate {}; + +template +struct ConvertImpl : ConvertImplUInt32Or64ToDate {}; + + /** Преобразование чисел, дат, дат-с-временем в строки: через форматирование. */ template void formatImpl(typename DataType::FieldType x, WriteBuffer & wb) { writeText(x, wb); } @@ -392,9 +451,9 @@ public: IDataType * from_type = &*block.getByPosition(arguments[0]).type; if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); - else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); - else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); - else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); + else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); + else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); + else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); @@ -403,7 +462,7 @@ public: else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); - else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); + else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); else if (typeid_cast(from_type)) ConvertImpl::execute(block, arguments, result); else throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), diff --git a/dbms/include/DB/Functions/FunctionsDictionaries.h b/dbms/include/DB/Functions/FunctionsDictionaries.h index 26383f44d3f..6db29c6b960 100644 --- a/dbms/include/DB/Functions/FunctionsDictionaries.h +++ b/dbms/include/DB/Functions/FunctionsDictionaries.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include @@ -877,6 +879,8 @@ DECLARE_DICT_GET_TRAITS(Int32, DataTypeInt32) DECLARE_DICT_GET_TRAITS(Int64, DataTypeInt64) DECLARE_DICT_GET_TRAITS(Float32, DataTypeFloat32) DECLARE_DICT_GET_TRAITS(Float64, DataTypeFloat64) +DECLARE_DICT_GET_TRAITS(UInt16, DataTypeDate) +DECLARE_DICT_GET_TRAITS(UInt32, DataTypeDateTime) #undef DECLARE_DICT_GET_TRAITS template @@ -1009,7 +1013,7 @@ private: }; template -const std::string FunctionDictGet::name = "dictGet" + TypeName::get(); +const std::string FunctionDictGet::name = "dictGet" + DataType{}.getName(); using FunctionDictGetUInt8 = FunctionDictGet; @@ -1022,6 +1026,8 @@ using FunctionDictGetInt32 = FunctionDictGet; using FunctionDictGetInt64 = FunctionDictGet; using FunctionDictGetFloat32 = FunctionDictGet; using FunctionDictGetFloat64 = FunctionDictGet; +using FunctionDictGetDate = FunctionDictGet; +using FunctionDictGetDateTime = FunctionDictGet; class FunctionDictGetHierarchy final : public IFunction @@ -1396,4 +1402,4 @@ private: }; -} +}; diff --git a/dbms/include/DB/IO/BufferBase.h b/dbms/include/DB/IO/BufferBase.h index a3afde8eb99..a986e3218bb 100644 --- a/dbms/include/DB/IO/BufferBase.h +++ b/dbms/include/DB/IO/BufferBase.h @@ -64,6 +64,9 @@ public: pos = ptr + offset; } + /// получить буфер + inline Buffer & internalBuffer() { return internal_buffer; } + /// получить часть буфера, из которого можно читать / в который можно писать данные inline Buffer & buffer() { return working_buffer; } diff --git a/dbms/include/DB/IO/BufferWithOwnMemory.h b/dbms/include/DB/IO/BufferWithOwnMemory.h index 09034322bcc..22cdabb1459 100644 --- a/dbms/include/DB/IO/BufferWithOwnMemory.h +++ b/dbms/include/DB/IO/BufferWithOwnMemory.h @@ -91,20 +91,26 @@ private: if (current_memory_tracker) current_memory_tracker->alloc(m_capacity); + char * new_m_data = nullptr; + if (!alignment) { - m_data = reinterpret_cast(malloc(m_capacity)); + new_m_data = reinterpret_cast(malloc(m_capacity)); - if (!m_data) + if (!new_m_data) throw Exception("Cannot allocate memory (malloc)", ErrorCodes::CANNOT_ALLOCATE_MEMORY); + m_data = new_m_data; + return; } - int res = posix_memalign(reinterpret_cast(&m_data), alignment, (m_capacity + alignment - 1) / alignment * alignment); + int res = posix_memalign(reinterpret_cast(&new_m_data), alignment, (m_capacity + alignment - 1) / alignment * alignment); if (0 != res) DB::throwFromErrno("Cannot allocate memory (posix_memalign)", ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + + m_data = new_m_data; } void dealloc() diff --git a/dbms/include/DB/IO/CachedCompressedReadBuffer.h b/dbms/include/DB/IO/CachedCompressedReadBuffer.h index 31b748ed2cc..2617d84dfe7 100644 --- a/dbms/include/DB/IO/CachedCompressedReadBuffer.h +++ b/dbms/include/DB/IO/CachedCompressedReadBuffer.h @@ -53,12 +53,13 @@ private: owned_cell.reset(new UncompressedCacheCell); size_t size_decompressed; - owned_cell->compressed_size = readCompressedData(size_decompressed); + size_t size_compressed_without_checksum; + owned_cell->compressed_size = readCompressedData(size_decompressed, size_compressed_without_checksum); if (owned_cell->compressed_size) { owned_cell->data.resize(size_decompressed); - decompress(owned_cell->data.m_data, size_decompressed); + decompress(owned_cell->data.m_data, size_decompressed, size_compressed_without_checksum); /// Положим данные в кэш. cache->set(key, owned_cell); diff --git a/dbms/include/DB/IO/CompressedReadBuffer.h b/dbms/include/DB/IO/CompressedReadBuffer.h index a359b2db162..7a3a919fa38 100644 --- a/dbms/include/DB/IO/CompressedReadBuffer.h +++ b/dbms/include/DB/IO/CompressedReadBuffer.h @@ -14,14 +14,15 @@ private: bool nextImpl() { size_t size_decompressed; - size_compressed = readCompressedData(size_decompressed); + size_t size_compressed_without_checksum; + size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum); if (!size_compressed) return false; memory.resize(size_decompressed); working_buffer = Buffer(&memory[0], &memory[size_decompressed]); - decompress(working_buffer.begin(), size_decompressed); + decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum); return true; } @@ -44,14 +45,15 @@ public: while (bytes_read < n) { size_t size_decompressed; + size_t size_compressed_without_checksum; - if (!readCompressedData(size_decompressed)) + if (!readCompressedData(size_decompressed, size_compressed_without_checksum)) return bytes_read; /// Если разжатый блок помещается целиком туда, куда его надо скопировать. if (size_decompressed <= n - bytes_read) { - decompress(to + bytes_read, size_decompressed); + decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum); bytes_read += size_decompressed; bytes += size_decompressed; } @@ -62,7 +64,7 @@ public: working_buffer = Buffer(&memory[0], &memory[size_decompressed]); pos = working_buffer.begin(); - decompress(working_buffer.begin(), size_decompressed); + decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum); bytes_read += read(to + bytes_read, n - bytes_read); break; diff --git a/dbms/include/DB/IO/CompressedReadBufferBase.h b/dbms/include/DB/IO/CompressedReadBufferBase.h index fbf031a27f7..3f38879f879 100644 --- a/dbms/include/DB/IO/CompressedReadBufferBase.h +++ b/dbms/include/DB/IO/CompressedReadBufferBase.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -32,7 +33,7 @@ protected: /// Прочитать сжатые данные в compressed_buffer. Достать из их заголовка размер разжатых данных. Проверить чексумму. /// Возвращает количество прочитанных байт. - size_t readCompressedData(size_t & size_decompressed) + size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum) { if (compressed_in->eof()) return 0; @@ -44,14 +45,15 @@ protected: compressed_in->readStrict(&own_compressed_buffer[0], QUICKLZ_HEADER_SIZE); UInt8 method = own_compressed_buffer[0]; /// См. CompressedWriteBuffer.h - size_t size_compressed; + + size_t & size_compressed = size_compressed_without_checksum; if (method < 0x80) { size_compressed = qlz_size_compressed(&own_compressed_buffer[0]); size_decompressed = qlz_size_decompressed(&own_compressed_buffer[0]); } - else if (method == 0x82) + else if (method == static_cast(CompressionMethodByte::LZ4) || method == static_cast(CompressionMethodByte::ZSTD)) { size_compressed = *reinterpret_cast(&own_compressed_buffer[1]); size_decompressed = *reinterpret_cast(&own_compressed_buffer[5]); @@ -85,7 +87,7 @@ protected: return size_compressed + sizeof(checksum); } - void decompress(char * to, size_t size_decompressed) + void decompress(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) { ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); @@ -99,10 +101,19 @@ protected: qlz_decompress(&compressed_buffer[0], to, qlz_state); } - else if (method == 0x82) + else if (method == static_cast(CompressionMethodByte::LZ4)) { if (LZ4_decompress_fast(&compressed_buffer[QUICKLZ_HEADER_SIZE], to, size_decompressed) < 0) - throw Exception("Cannot LZ4_decompress_fast", ErrorCodes::CORRUPTED_DATA); + throw Exception("Cannot LZ4_decompress_fast", ErrorCodes::CANNOT_DECOMPRESS); + } + else if (method == static_cast(CompressionMethodByte::ZSTD)) + { + size_t res = ZSTD_decompress( + to, size_decompressed, + &compressed_buffer[QUICKLZ_HEADER_SIZE], size_compressed_without_checksum - QUICKLZ_HEADER_SIZE); + + if (ZSTD_isError(res)) + throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS); } else throw Exception("Unknown compression method: " + toString(method), ErrorCodes::UNKNOWN_COMPRESSION_METHOD); diff --git a/dbms/include/DB/IO/CompressedReadBufferFromFile.h b/dbms/include/DB/IO/CompressedReadBufferFromFile.h index fa9e0552a57..f9fbd51344c 100644 --- a/dbms/include/DB/IO/CompressedReadBufferFromFile.h +++ b/dbms/include/DB/IO/CompressedReadBufferFromFile.h @@ -24,14 +24,15 @@ private: bool nextImpl() { size_t size_decompressed; - size_compressed = readCompressedData(size_decompressed); + size_t size_compressed_without_checksum; + size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum); if (!size_compressed) return false; memory.resize(size_decompressed); working_buffer = Buffer(&memory[0], &memory[size_decompressed]); - decompress(working_buffer.begin(), size_decompressed); + decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum); return true; } @@ -81,8 +82,9 @@ public: while (bytes_read < n) { size_t size_decompressed = 0; + size_t size_compressed_without_checksum = 0; - size_t new_size_compressed = readCompressedData(size_decompressed); + size_t new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum); size_compressed = 0; /// file_in больше не указывает на конец блока в working_buffer. if (!new_size_compressed) return bytes_read; @@ -90,7 +92,7 @@ public: /// Если разжатый блок помещается целиком туда, куда его надо скопировать. if (size_decompressed <= n - bytes_read) { - decompress(to + bytes_read, size_decompressed); + decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum); bytes_read += size_decompressed; bytes += size_decompressed; } @@ -102,7 +104,7 @@ public: working_buffer = Buffer(&memory[0], &memory[size_decompressed]); pos = working_buffer.begin(); - decompress(working_buffer.begin(), size_decompressed); + decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum); bytes_read += read(to + bytes_read, n - bytes_read); break; diff --git a/dbms/include/DB/IO/CompressedStream.h b/dbms/include/DB/IO/CompressedStream.h index 6d5e921d398..39364881c65 100644 --- a/dbms/include/DB/IO/CompressedStream.h +++ b/dbms/include/DB/IO/CompressedStream.h @@ -11,15 +11,48 @@ namespace DB { -namespace CompressionMethod +/** Метод сжатия */ +enum class CompressionMethod { - /** Метод сжатия */ - enum Enum - { - QuickLZ, - LZ4, - LZ4HC, /// Формат такой же, как у LZ4. Разница только при сжатии. - }; -} + QuickLZ, + LZ4, + LZ4HC, /// Формат такой же, как у LZ4. Разница только при сжатии. + ZSTD, /// Экспериментальный алгоритм: https://github.com/Cyan4973/zstd +}; + +/** Формат сжатого блока следующий: + * + * Первые 16 байт - чексумма от всех остальных байт блока. Сейчас используется только CityHash128. + * В дальнейшем можно предусмотреть другие чексуммы, хотя сделать их другого размера не получится. + * + * Следующий байт определяет алгоритм сжатия. Далее всё зависит от алгоритма. + * + * Первые 4 варианта совместимы с QuickLZ level 1. + * То есть, если значение первого байта < 4, для разжатия достаточно использовать функцию qlz_level1_decompress. + * + * 0x00 - несжатые данные, маленький блок. Далее один байт - размер сжатых данных, с учётом заголовка; один байт - размер несжатых данных. + * 0x01 - сжатые данные, QuickLZ level 1, маленький блок. Далее два байта аналогично. + * 0x02 - несжатые данные, большой блок. Далее 4 байта - размер сжатых данных, с учётом заголовка; 4 байта - размер несжатых данных. + * 0x03 - сжатые данные, QuickLZ level 1, большой блок. Далее 8 байт аналогично. + * + * 0x82 - LZ4 или LZ4HC (они имеют одинаковый формат). + * Далее 4 байта - размер сжатых данных, с учётом заголовка; 4 байта - размер несжатых данных. + * + * NOTE: Почему 0x82? + * Изначально использовался только QuickLZ. Потом был добавлен LZ4. + * Старший бит выставлен, чтобы отличить от QuickLZ, а второй бит выставлен для совместимости, + * чтобы работали функции qlz_size_compressed, qlz_size_decompressed. + * Хотя сейчас такая совместимость уже не актуальна. + * + * 0x90 - ZSTD + * + * Все размеры - little endian. + */ + +enum class CompressionMethodByte : uint8_t +{ + LZ4 = 0x82, + ZSTD = 0x90, +}; } diff --git a/dbms/include/DB/IO/CompressedWriteBuffer.h b/dbms/include/DB/IO/CompressedWriteBuffer.h index 36b72693287..3065defd1aa 100644 --- a/dbms/include/DB/IO/CompressedWriteBuffer.h +++ b/dbms/include/DB/IO/CompressedWriteBuffer.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -24,7 +25,7 @@ class CompressedWriteBuffer : public BufferWithOwnMemory { private: WriteBuffer & out; - CompressionMethod::Enum method; + CompressionMethod method; PODArray compressed_buffer; qlz_state_compress * qlz_state; @@ -38,31 +39,7 @@ private: size_t compressed_size = 0; char * compressed_buffer_ptr = nullptr; - /** Формат сжатого блока следующий: - * - * Первые 16 байт - чексумма от всех остальных байт блока. Сейчас используется только CityHash128. - * В дальнейшем можно предусмотреть другие чексуммы, хотя сделать их другого размера не получится. - * - * Следующий байт определяет алгоритм сжатия. Далее всё зависит от алгоритма. - * - * Первые 4 варианта совместимы с QuickLZ level 1. - * То есть, если значение первого байта < 4, для разжатия достаточно использовать функцию qlz_level1_decompress. - * - * 0x00 - несжатые данные, маленький блок. Далее один байт - размер сжатых данных, с учётом заголовка; один байт - размер несжатых данных. - * 0x01 - сжатые данные, QuickLZ level 1, маленький блок. Далее два байта аналогично. - * 0x02 - несжатые данные, большой блок. Далее 4 байта - размер сжатых данных, с учётом заголовка; 4 байта - размер несжатых данных. - * 0x03 - сжатые данные, QuickLZ level 1, большой блок. Далее 8 байт аналогично. - * - * 0x82 - LZ4 или LZ4HC (они имеют одинаковый формат). - * Далее 4 байта - размер сжатых данных, с учётом заголовка; 4 байта - размер несжатых данных. - * - * NOTE: Почему 0x82? - * Изначально использовался только QuickLZ. Потом был добавлен LZ4. - * Старший бит выставлен, чтобы отличить от QuickLZ, а второй бит выставлен для совместимости, - * чтобы работали функции qlz_size_compressed, qlz_size_decompressed. - * Хотя сейчас такая совместимость уже не актуальна. - * - * Все размеры - little endian. + /** Формат сжатого блока - см. CompressedStream.h */ switch (method) @@ -88,7 +65,7 @@ private: compressed_buffer.resize(header_size + LZ4_COMPRESSBOUND(uncompressed_size)); - compressed_buffer[0] = 0x82; /// Второй бит - для совместимости с QuickLZ - обозначает, что размеры записываются 4 байтами. + compressed_buffer[0] = static_cast(CompressionMethodByte::LZ4); if (method == CompressionMethod::LZ4) compressed_size = header_size + LZ4_compress( @@ -110,6 +87,34 @@ private: compressed_buffer_ptr = &compressed_buffer[0]; break; } + case CompressionMethod::ZSTD: + { + static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); + + compressed_buffer.resize(header_size + ZSTD_compressBound(uncompressed_size)); + + compressed_buffer[0] = static_cast(CompressionMethodByte::ZSTD); + + size_t res = ZSTD_compress( + &compressed_buffer[header_size], + compressed_buffer.size(), + working_buffer.begin(), + uncompressed_size); + + if (ZSTD_isError(res)) + throw Exception("Cannot compress block with ZSTD: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_COMPRESS); + + compressed_size = header_size + res; + + UInt32 compressed_size_32 = compressed_size; + UInt32 uncompressed_size_32 = uncompressed_size; + + memcpy(&compressed_buffer[1], reinterpret_cast(&compressed_size_32), sizeof(compressed_size_32)); + memcpy(&compressed_buffer[5], reinterpret_cast(&uncompressed_size_32), sizeof(uncompressed_size_32)); + + compressed_buffer_ptr = &compressed_buffer[0]; + break; + } default: throw Exception("Unknown compression method", ErrorCodes::UNKNOWN_COMPRESSION_METHOD); } @@ -123,7 +128,7 @@ private: public: CompressedWriteBuffer( WriteBuffer & out_, - CompressionMethod::Enum method_ = CompressionMethod::LZ4, + CompressionMethod method_ = CompressionMethod::LZ4, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : BufferWithOwnMemory(buf_size), out(out_), method(method_), qlz_state(new qlz_state_compress) {} diff --git a/dbms/include/DB/IO/ReadBufferAIO.h b/dbms/include/DB/IO/ReadBufferAIO.h new file mode 100644 index 00000000000..09578707b3c --- /dev/null +++ b/dbms/include/DB/IO/ReadBufferAIO.h @@ -0,0 +1,69 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +/** Класс для асинхронной чтения данных. + * Все размеры и смещения должны быть кратны DEFAULT_AIO_FILE_BLOCK_SIZE байтам. + */ +class ReadBufferAIO : public BufferWithOwnMemory +{ +public: + ReadBufferAIO(const std::string & filename_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags_ = -1, mode_t mode_ = 0666, + char * existing_memory_ = nullptr); + ~ReadBufferAIO() override; + + ReadBufferAIO(const ReadBufferAIO &) = delete; + ReadBufferAIO & operator=(const ReadBufferAIO &) = delete; + + void setMaxBytes(size_t max_bytes_read_); + off_t seek(off_t off, int whence = SEEK_SET); + off_t getPositionInFile(); + std::string getFileName() const noexcept { return filename; } + int getFD() const noexcept { return fd; } + +private: + off_t getPositionInFileRelaxed() const noexcept; + bool nextImpl(); + /// Ждать окончания текущей асинхронной задачи. + void waitForAIOCompletion(); + /// Менять местами основной и дублирующий буферы. + void swapBuffers() noexcept; + +private: + /// Буфер для асинхронных операций чтения данных. + BufferWithOwnMemory fill_buffer; + + iocb request; + std::vector request_ptrs; + std::vector events; + + AIOContext aio_context; + + const std::string filename; + + size_t max_bytes_read = std::numeric_limits::max(); + size_t total_bytes_read = 0; + off_t pos_in_file = 0; + int fd = -1; + + /// Асинхронная операция чтения ещё не завершилась. + bool is_pending_read = false; + /// Было получено исключение. + bool got_exception = false; + /// Конец файла достигнут. + bool is_eof = false; + /// Был отправлен хоть один запрос на асинхронную операцию чтения. + bool is_started = false; +}; + +} diff --git a/dbms/include/DB/IO/ReadBufferFromFileDescriptor.h b/dbms/include/DB/IO/ReadBufferFromFileDescriptor.h index 1a2febf61b5..b412055f509 100644 --- a/dbms/include/DB/IO/ReadBufferFromFileDescriptor.h +++ b/dbms/include/DB/IO/ReadBufferFromFileDescriptor.h @@ -99,7 +99,7 @@ public: } } - size_t getPositionInFile() + off_t getPositionInFile() { return pos_in_file - (working_buffer.end() - pos); } diff --git a/dbms/include/DB/IO/WriteBufferAIO.h b/dbms/include/DB/IO/WriteBufferAIO.h new file mode 100644 index 00000000000..732a45ec588 --- /dev/null +++ b/dbms/include/DB/IO/WriteBufferAIO.h @@ -0,0 +1,65 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +/** Класс для асинхронной записи данных. + * Все размеры и смещения должны быть кратны DEFAULT_AIO_FILE_BLOCK_SIZE байтам. + */ +class WriteBufferAIO : public BufferWithOwnMemory +{ +public: + WriteBufferAIO(const std::string & filename_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags_ = -1, mode_t mode_ = 0666, + char * existing_memory_ = nullptr); + ~WriteBufferAIO() override; + + WriteBufferAIO(const WriteBufferAIO &) = delete; + WriteBufferAIO & operator=(const WriteBufferAIO &) = delete; + + off_t seek(off_t off, int whence = SEEK_SET); + off_t getPositionInFile(); + void truncate(off_t length = 0); + void sync(); + std::string getFileName() const noexcept { return filename; } + int getFD() const noexcept { return fd; } + +private: + /// Если в буфере ещё остались данные - запишем их. + void flush(); + /// + void nextImpl(); + /// Ждать окончания текущей асинхронной задачи. + void waitForAIOCompletion(); + /// Менять местами основной и дублирующий буферы. + void swapBuffers() noexcept; + +private: + /// Буфер для асинхронных операций записи данных. + BufferWithOwnMemory flush_buffer; + + iocb request; + std::vector request_ptrs; + std::vector events; + + AIOContext aio_context; + + const std::string filename; + + off_t pos_in_file = 0; + int fd = -1; + + /// Асинхронная операция записи ещё не завершилась. + bool is_pending_write = false; + /// Было получено исключение. + bool got_exception = false; +}; + +} diff --git a/dbms/include/DB/Interpreters/Aggregator.h b/dbms/include/DB/Interpreters/Aggregator.h index 771c58f9c61..5b6bca108af 100644 --- a/dbms/include/DB/Interpreters/Aggregator.h +++ b/dbms/include/DB/Interpreters/Aggregator.h @@ -528,9 +528,7 @@ struct AggregatedDataVariants : private boost::noncopyable void init(Type type_) { - type = type_; - - switch (type) + switch (type_) { case Type::EMPTY: break; case Type::without_key: break; @@ -543,6 +541,8 @@ struct AggregatedDataVariants : private boost::noncopyable default: throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } + + type = type_; } size_t size() const @@ -676,19 +676,6 @@ APPLY_FOR_AGGREGATED_VARIANTS(M) class Aggregator { public: - Aggregator(const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, - size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, Compiler * compiler_, UInt32 min_count_to_compile_, - size_t group_by_two_level_threshold_) - : keys(keys_), aggregates(aggregates_), aggregates_size(aggregates.size()), - overflow_row(overflow_row_), - max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_), - compiler(compiler_), min_count_to_compile(min_count_to_compile_), group_by_two_level_threshold(group_by_two_level_threshold_) - { - std::sort(keys.begin(), keys.end()); - keys.erase(std::unique(keys.begin(), keys.end()), keys.end()); - keys_size = keys.size(); - } - Aggregator(const Names & key_names_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_) @@ -739,6 +726,9 @@ public: /// Для IBlockInputStream. String getID() const; + size_t getNumberOfKeys() const { return keys_size; } + size_t getNumberOfAggregates() const { return aggregates_size; } + protected: friend struct AggregatedDataVariants; @@ -945,7 +935,7 @@ protected: template void mergeStreamsImpl( Block & block, - AggregatedDataVariants & result, + const Sizes & key_sizes, Arena * aggregates_pool, Method & method, Table & data) const; diff --git a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h index c4b01ffe0ba..24ba3c863e7 100644 --- a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h +++ b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h @@ -218,6 +218,9 @@ private: /// Eliminates injective function calls and constant expressions from group by statement void optimizeGroupBy(); + /// Удалить из ORDER BY повторяющиеся элементы. + void optimizeOrderBy(); + /// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn. void makeSet(ASTFunction * node, const Block & sample_block); diff --git a/dbms/include/DB/Interpreters/ExternalDictionaries.h b/dbms/include/DB/Interpreters/ExternalDictionaries.h index c067bdc78c7..971f6812f85 100644 --- a/dbms/include/DB/Interpreters/ExternalDictionaries.h +++ b/dbms/include/DB/Interpreters/ExternalDictionaries.h @@ -39,7 +39,10 @@ private: static const auto check_period_sec = 5; mutable std::mutex dictionaries_mutex; - std::unordered_map>> dictionaries; + + using dictionary_ptr_t = std::shared_ptr>; + using dictionary_origin_pair_t = std::pair; + std::unordered_map dictionaries; /// exception pointers for notifying user about failures on dictionary creation std::unordered_map stored_exceptions; std::unordered_map update_times; @@ -52,9 +55,10 @@ private: Logger * log; - Poco::Timestamp config_last_modified{0}; + std::unordered_map last_modification_times; void reloadImpl(); + void reloadFromFile(const std::string & config_path); void reloadPeriodically() { @@ -105,7 +109,7 @@ public: }; } - return it->second->get(); + return it->second.first->get(); } }; diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 3f886c921f8..063d12ff745 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.h @@ -109,6 +109,12 @@ struct Settings \ /** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \ M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \ + \ + /** Минимальное количество байтов для операций ввода/ввывода минуя кэш страниц */ \ + M(SettingUInt64, min_bytes_to_use_direct_io, (20U * 1024U * 1024U * 1024U)) \ + \ + /** Кидать исключение, если есть индекс по дате, и он не используется. */ \ + M(SettingBool, force_index_by_date, 0) \ /// Всевозможные ограничения на выполнение запроса. Limits limits; diff --git a/dbms/include/DB/Interpreters/SpecializedAggregator.h b/dbms/include/DB/Interpreters/SpecializedAggregator.h index 7830f5f8da2..5ef0c91f59f 100644 --- a/dbms/include/DB/Interpreters/SpecializedAggregator.h +++ b/dbms/include/DB/Interpreters/SpecializedAggregator.h @@ -235,11 +235,11 @@ void NO_INLINE Aggregator::executeSpecializedCase( /// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом. if (inserted) { + AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); + aggregate_data = nullptr; + method.onNewKey(*it, keys_size, i, keys, *aggregates_pool); - AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); - - aggregate_data = nullptr; AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states); AggregateFunctionsList::forEach(AggregateFunctionsCreator( diff --git a/dbms/include/DB/Storages/IStorage.h b/dbms/include/DB/Storages/IStorage.h index aeed983161d..158c2d6ccd2 100644 --- a/dbms/include/DB/Storages/IStorage.h +++ b/dbms/include/DB/Storages/IStorage.h @@ -47,29 +47,19 @@ public: /// Основное имя типа таблицы (например, StorageMergeTree). virtual std::string getName() const = 0; - /** Возвращает true, если хранилище получает данные с удалённого сервера или серверов. - */ + /** Возвращает true, если хранилище получает данные с удалённого сервера или серверов. */ virtual bool isRemote() const { return false; } - virtual void storeExternalTables(const std::map & tables_) - { - throw Exception("Method storeExternalTables is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - /** Возвращает true, если хранилище поддерживает запросы с секцией SAMPLE. - */ + /** Возвращает true, если хранилище поддерживает запросы с секцией SAMPLE. */ virtual bool supportsSampling() const { return false; } - /** Возвращает true, если хранилище поддерживает запросы с секцией FINAL. - */ + /** Возвращает true, если хранилище поддерживает запросы с секцией FINAL. */ virtual bool supportsFinal() const { return false; } - /** Возвращает true, если хранилище поддерживает запросы с секцией PREWHERE. - */ + /** Возвращает true, если хранилище поддерживает запросы с секцией PREWHERE. */ virtual bool supportsPrewhere() const { return false; } - /** Возвращает true, если хранилище поддерживает несколько реплик. - */ + /** Возвращает true, если хранилище поддерживает несколько реплик. */ virtual bool supportsParallelReplicas() const { return false; } /** Не дает изменять описание таблицы (в том числе переименовывать и удалять таблицу). diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h index d2fac675c8a..6810c9f5695 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h @@ -55,6 +55,8 @@ namespace DB * (см. CollapsingSortedBlockInputStream.h) * - Summing - при склейке кусков, при совпадении PK суммировать все числовые столбцы, не входящие в PK. * - Aggregating - при склейке кусков, при совпадении PK, делается слияние состояний столбцов-агрегатных функций. + * - Unsorted - при склейке кусков, данные не упорядочиваются, а всего лишь конкатенируются; + * - это позволяет читать данные ровно такими пачками, какими они были записаны. */ /** Этот класс хранит список кусков и параметры структуры данных. @@ -399,18 +401,22 @@ public: } size_t key_size = storage.sort_descr.size(); - index.resize(key_size * size); - String index_path = storage.full_path + name + "/primary.idx"; - ReadBufferFromFile index_file(index_path, - std::min(static_cast(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(index_path).getSize())); + if (key_size) + { + index.resize(key_size * size); - for (size_t i = 0; i < size; ++i) - for (size_t j = 0; j < key_size; ++j) - storage.primary_key_sample.getByPosition(j).type->deserializeBinary(index[i * key_size + j], index_file); + String index_path = storage.full_path + name + "/primary.idx"; + ReadBufferFromFile index_file(index_path, + std::min(static_cast(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(index_path).getSize())); - if (!index_file.eof()) - throw Exception("index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE); + for (size_t i = 0; i < size; ++i) + for (size_t j = 0; j < key_size; ++j) + storage.primary_key_sample.getByPosition(j).type->deserializeBinary(index[i * key_size + j], index_file); + + if (!index_file.eof()) + throw Exception("index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE); + } size_in_bytes = calcTotalSize(storage.full_path + name + "/"); } @@ -468,7 +474,7 @@ public: if (!checksums.empty()) { - if (!checksums.files.count("primary.idx")) + if (!storage.sort_descr.empty() && !checksums.files.count("primary.idx")) throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART); if (require_part_metadata) @@ -486,12 +492,14 @@ public: } else { - /// Проверяем, что первичный ключ непуст. + if (!storage.sort_descr.empty()) + { + /// Проверяем, что первичный ключ непуст. + Poco::File index_file(path + "/primary.idx"); - Poco::File index_file(path + "/primary.idx"); - - if (!index_file.exists() || index_file.getSize() == 0) - throw Exception("Part " + path + " is broken: primary key is empty.", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); + if (!index_file.exists() || index_file.getSize() == 0) + throw Exception("Part " + path + " is broken: primary key is empty.", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); + } /// Проверяем, что все засечки непусты и имеют одинаковый размер. @@ -620,6 +628,7 @@ public: Collapsing, Summing, Aggregating, + Unsorted, }; static void doNothing(const String & name) {} @@ -628,7 +637,7 @@ public: * (корректность имён и путей не проверяется) * состоящую из указанных столбцов. * - * primary_expr_ast - выражение для сортировки; + * primary_expr_ast - выражение для сортировки; Пустое для UnsortedMergeTree. * date_column_name - имя столбца с датой; * index_granularity - на сколько строчек пишется одно значение индекса. * require_part_metadata - обязательно ли в директории с куском должны быть checksums.txt и columns.txt diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index b30aeef2fe7..9f78b318122 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -128,8 +128,11 @@ public: } catch (const Exception & e) { - if (e.code() != ErrorCodes::ALL_REQUESTED_COLUMNS_ARE_MISSING) + if (e.code() != ErrorCodes::ALL_REQUESTED_COLUMNS_ARE_MISSING + && e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) + { storage.reportBrokenPart(part_name); + } /// Более хорошая диагностика. throw Exception(e.message() + "\n(while reading from part " + path + " from mark " + toString(from_mark) + " to " @@ -258,8 +261,10 @@ public: /// remove added column to ensure same content among all blocks if (added_column) { + res.erase(0); streams.erase(added_column->name); columns.erase(std::begin(columns)); + added_column = nullptr; } /// sort columns to ensure consistent order among all blocks @@ -280,12 +285,6 @@ public: std::swap(res, ordered_block); } - else if (added_column) - { - res.erase(0); - } - - added_column = nullptr; } catch (const Exception & e) { diff --git a/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h index 2647ce2788b..7a537dd2a7a 100644 --- a/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h @@ -15,18 +15,32 @@ namespace DB class IMergedBlockOutputStream : public IBlockOutputStream { public: - IMergedBlockOutputStream(MergeTreeData & storage_, size_t min_compress_block_size_, size_t max_compress_block_size_) : storage(storage_), index_offset(0), min_compress_block_size(min_compress_block_size_), max_compress_block_size(max_compress_block_size_) + IMergedBlockOutputStream( + MergeTreeData & storage_, + size_t min_compress_block_size_, + size_t max_compress_block_size_, + CompressionMethod compression_method_) + : storage(storage_), + min_compress_block_size(min_compress_block_size_), + max_compress_block_size(max_compress_block_size_), + compression_method(compression_method_) { } protected: - typedef std::set OffsetColumns; + using OffsetColumns = std::set; + struct ColumnStream { - ColumnStream(const String & escaped_column_name_, const String & data_path, const std::string & marks_path, size_t max_compress_block_size = DEFAULT_MAX_COMPRESS_BLOCK_SIZE) : + ColumnStream( + const String & escaped_column_name_, + const String & data_path, + const std::string & marks_path, + size_t max_compress_block_size, + CompressionMethod compression_method) : escaped_column_name(escaped_column_name_), plain_file(data_path, max_compress_block_size, O_TRUNC | O_CREAT | O_WRONLY), - plain_hashing(plain_file), compressed_buf(plain_hashing), compressed(compressed_buf), + plain_hashing(plain_file), compressed_buf(plain_hashing, compression_method), compressed(compressed_buf), marks_file(marks_path, 4096, O_TRUNC | O_CREAT | O_WRONLY), marks(marks_file) {} String escaped_column_name; @@ -70,7 +84,7 @@ protected: } }; - typedef std::map > ColumnStreams; + using ColumnStreams = std::map>; void addStream(const String & path, const String & name, const IDataType & type, size_t level = 0, String filename = "") { @@ -92,7 +106,8 @@ protected: escaped_size_name, path + escaped_size_name + ".bin", path + escaped_size_name + ".mrk", - max_compress_block_size)); + max_compress_block_size, + compression_method)); addStream(path, name, *type_arr->getNestedType(), level + 1); } @@ -101,7 +116,8 @@ protected: escaped_column_name, path + escaped_column_name + ".bin", path + escaped_column_name + ".mrk", - max_compress_block_size)); + max_compress_block_size, + compression_method)); } @@ -146,7 +162,8 @@ protected: type_arr->serializeOffsets(column, stream.compressed, prev_mark, limit); - stream.compressed.nextIfAtEnd(); /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего. + /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего. + stream.compressed.nextIfAtEnd(); prev_mark += limit; } @@ -180,7 +197,8 @@ protected: type.serializeBinary(column, stream.compressed, prev_mark, limit); - stream.compressed.nextIfAtEnd(); /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего. + /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего. + stream.compressed.nextIfAtEnd(); prev_mark += limit; } @@ -192,10 +210,12 @@ protected: ColumnStreams column_streams; /// Смещение до первой строчки блока, для которой надо записать индекс. - size_t index_offset; + size_t index_offset = 0; size_t min_compress_block_size; size_t max_compress_block_size; + + CompressionMethod compression_method; }; /** Для записи одного куска. Данные уже отсортированы, относятся к одному месяцу, и пишутся в один кускок. @@ -203,13 +223,23 @@ protected: class MergedBlockOutputStream : public IMergedBlockOutputStream { public: - MergedBlockOutputStream(MergeTreeData & storage_, String part_path_, const NamesAndTypesList & columns_list_) - : IMergedBlockOutputStream(storage_, storage_.context.getSettings().min_compress_block_size, storage_.context.getSettings().max_compress_block_size), columns_list(columns_list_), part_path(part_path_), marks_count(0) + MergedBlockOutputStream( + MergeTreeData & storage_, + String part_path_, + const NamesAndTypesList & columns_list_, + CompressionMethod compression_method) + : IMergedBlockOutputStream( + storage_, storage_.context.getSettings().min_compress_block_size, + storage_.context.getSettings().max_compress_block_size, compression_method), + columns_list(columns_list_), part_path(part_path_) { Poco::File(part_path).createDirectories(); - index_file_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); - index_stream = new HashingWriteBuffer(*index_file_stream); + if (storage.mode != MergeTreeData::Unsorted) + { + index_file_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); + index_stream = new HashingWriteBuffer(*index_file_stream); + } for (const auto & it : columns_list) addStream(part_path, it.name, *it.type); @@ -233,7 +263,9 @@ public: { for (PrimaryColumns::const_iterator it = primary_columns.begin(); it != primary_columns.end(); ++it) { - index_vec.push_back((*(*it)->column)[i]); + if (storage.mode != MergeTreeData::Unsorted) + index_vec.push_back((*(*it)->column)[i]); + (*it)->type->serializeBinary(index_vec.back(), *index_stream); } @@ -264,9 +296,13 @@ public: /// Заканчиваем запись и достаем чексуммы. MergeTreeData::DataPart::Checksums checksums; - index_stream->next(); - checksums.files["primary.idx"].file_size = index_stream->count(); - checksums.files["primary.idx"].file_hash = index_stream->getHash(); + if (storage.mode != MergeTreeData::Unsorted) + { + index_stream->next(); + checksums.files["primary.idx"].file_size = index_stream->count(); + checksums.files["primary.idx"].file_hash = index_stream->getHash(); + index_stream = nullptr; + } for (ColumnStreams::iterator it = column_streams.begin(); it != column_streams.end(); ++it) { @@ -274,7 +310,6 @@ public: it->second->addToChecksums(checksums); } - index_stream = nullptr; column_streams.clear(); if (marks_count == 0) @@ -315,7 +350,7 @@ private: NamesAndTypesList columns_list; String part_path; - size_t marks_count; + size_t marks_count = 0; SharedPtr index_file_stream; SharedPtr index_stream; @@ -328,8 +363,11 @@ typedef Poco::SharedPtr MergedBlockOutputStreamPtr; class MergedColumnOnlyOutputStream : public IMergedBlockOutputStream { public: - MergedColumnOnlyOutputStream(MergeTreeData & storage_, String part_path_, bool sync_ = false) : - IMergedBlockOutputStream(storage_, storage_.context.getSettings().min_compress_block_size, storage_.context.getSettings().max_compress_block_size), part_path(part_path_), initialized(false), sync(sync_) + MergedColumnOnlyOutputStream(MergeTreeData & storage_, String part_path_, bool sync_, CompressionMethod compression_method) + : IMergedBlockOutputStream( + storage_, storage_.context.getSettings().min_compress_block_size, + storage_.context.getSettings().max_compress_block_size, compression_method), + part_path(part_path_), sync(sync_) { } @@ -386,8 +424,7 @@ public: private: String part_path; - bool initialized; - + bool initialized = false; bool sync; }; diff --git a/dbms/include/DB/Storages/StorageChunkMerger.h b/dbms/include/DB/Storages/StorageChunkMerger.h index 4d864d81a67..eaad94e428a 100644 --- a/dbms/include/DB/Storages/StorageChunkMerger.h +++ b/dbms/include/DB/Storages/StorageChunkMerger.h @@ -53,25 +53,26 @@ public: ~StorageChunkMerger() override; private: - String this_database; - String name; + const String this_database; + const String name; NamesAndTypesListPtr columns; - String source_database; + const String source_database; OptimizedRegularExpression table_name_regexp; std::string destination_name_prefix; - size_t chunks_to_merge; + const size_t chunks_to_merge; Context & context; Settings settings; - std::thread merge_thread; - Poco::Event cancel_merge_thread; - Logger * log; - volatile bool shutdown_called; /// Название виртуального столбца, отвечающего за имя таблицы, из которой идет чтение. (Например "_table") String _table_column_name; + class MergeTask; + using MergeTaskPtr = std::shared_ptr; + MergeTaskPtr merge_task; + DB::BackgroundProcessingPool::TaskHandle merge_task_handle; + StorageChunkMerger( const std::string & this_database_, const std::string & name_, @@ -85,11 +86,6 @@ private: size_t chunks_to_merge_, Context & context_); - void mergeThread(); - bool maybeMergeSomething(); - Storages selectChunksToMerge(); - bool mergeChunks(const Storages & chunks); - Block getBlockWithVirtualColumns(const Storages & selected_tables) const; typedef std::set TableNames; diff --git a/dbms/include/DB/Storages/StorageDistributed.h b/dbms/include/DB/Storages/StorageDistributed.h index b3ed52c9f18..73f2f414b74 100644 --- a/dbms/include/DB/Storages/StorageDistributed.h +++ b/dbms/include/DB/Storages/StorageDistributed.h @@ -55,8 +55,6 @@ public: bool hasColumn(const String & column_name) const override; bool isRemote() const override { return true; } - /// Сохранить временные таблицы, чтобы при следующем вызове метода read переслать их на удаленные серверы. - void storeExternalTables(const Tables & tables_) override { external_tables = tables_; } BlockInputStreams read( const Names & column_names, @@ -121,10 +119,6 @@ private: Context & context; - /// Временные таблицы, которые необходимо отправить на сервер. Переменная очищается после каждого вызова метода read - /// Для подготовки к отправке нужно использовтаь метод storeExternalTables - Tables external_tables; - /// Используется только, если таблица должна владеть объектом Cluster, которым больше никто не владеет - для реализации TableFunctionRemote. SharedPtr owned_cluster; diff --git a/dbms/include/DB/Storages/StorageReplicatedMergeTree.h b/dbms/include/DB/Storages/StorageReplicatedMergeTree.h index 9f9f83b4c1c..588c2013a2b 100644 --- a/dbms/include/DB/Storages/StorageReplicatedMergeTree.h +++ b/dbms/include/DB/Storages/StorageReplicatedMergeTree.h @@ -13,7 +13,7 @@ #include #include #include -#include + namespace DB { diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 153d6600581..53d194a8510 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include @@ -230,7 +231,9 @@ static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const { - if (name == "count") + if (name == "debug") + return new AggregateFunctionDebug; + else if (name == "count") return new AggregateFunctionCount; else if (name == "any") return createAggregateFunctionSingleValue(name, argument_types); @@ -602,6 +605,7 @@ bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int { static const char * names[] { + "debug", "count", "any", "anyLast", diff --git a/dbms/src/Client/Benchmark.cpp b/dbms/src/Client/Benchmark.cpp index bad0d707e1a..c89630780bd 100644 --- a/dbms/src/Client/Benchmark.cpp +++ b/dbms/src/Client/Benchmark.cpp @@ -56,9 +56,9 @@ public: Benchmark(unsigned concurrency_, double delay_, const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, const Settings & settings_) - : concurrency(concurrency_), delay(delay_), queue(concurrency), pool(concurrency), + : concurrency(concurrency_), delay(delay_), queue(concurrency), connections(concurrency, host_, port_, default_database_, user_, password_, data_type_factory), - settings(settings_) + settings(settings_), pool(concurrency) { std::cerr << std::fixed << std::setprecision(3); @@ -78,8 +78,6 @@ private: typedef ConcurrentBoundedQueue Queue; Queue queue; - boost::threadpool::pool pool; - DataTypeFactory data_type_factory; ConnectionPool connections; Settings settings; @@ -123,6 +121,8 @@ private: Poco::FastMutex mutex; + boost::threadpool::pool pool; + void readQueries() { diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 9b498318b1a..f6b81893991 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -162,11 +162,37 @@ void Connection::forceConnected() } } - +struct PingTimeoutSetter +{ + PingTimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & ping_timeout_) + : socket(socket_), ping_timeout(ping_timeout_) + { + old_send_timeout = socket.getSendTimeout(); + old_receive_timeout = socket.getReceiveTimeout(); + + if (old_send_timeout > ping_timeout) + socket.setSendTimeout(ping_timeout); + if (old_receive_timeout > ping_timeout) + socket.setReceiveTimeout(ping_timeout); + } + + ~PingTimeoutSetter() + { + socket.setSendTimeout(old_send_timeout); + socket.setReceiveTimeout(old_receive_timeout); + } + + Poco::Net::StreamSocket & socket; + Poco::Timespan ping_timeout; + Poco::Timespan old_send_timeout; + Poco::Timespan old_receive_timeout; +}; + bool Connection::ping() { // LOG_TRACE(log_wrapper.get(), "Ping (" << getServerAddress() << ")"); + PingTimeoutSetter timeout_setter(socket, ping_timeout); try { UInt64 pong = 0; diff --git a/dbms/src/Core/Exception.cpp b/dbms/src/Core/Exception.cpp index b5e47069934..2571762489e 100644 --- a/dbms/src/Core/Exception.cpp +++ b/dbms/src/Core/Exception.cpp @@ -43,6 +43,14 @@ ExceptionPtr cloneCurrentException() } } +inline std::string demangle(const char * const mangled, int & status) +{ + const auto demangled_str = abi::__cxa_demangle(mangled, 0, 0, &status); + std::string demangled{demangled_str}; + free(demangled_str); + + return demangled; +} void tryLogCurrentException(const char * log_name) { @@ -72,10 +80,8 @@ void tryLogCurrentException(const char * log_name) { try { - int status; - char * realname = abi::__cxa_demangle(typeid(e).name(), 0, 0, &status); - std::string name = realname; - free(realname); + int status = 0; + auto name = demangle(typeid(e).name(), status); if (status) name += " (demangling status: " + toString(status) + ")"; @@ -88,7 +94,13 @@ void tryLogCurrentException(const char * log_name) { try { - LOG_ERROR(&Logger::get(log_name), "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION); + int status = 0; + auto name = demangle(abi::__cxa_current_exception_type()->name(), status); + + if (status) + name += " (demangling status: " + toString(status) + ")"; + + LOG_ERROR(&Logger::get(log_name), "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name); } catch (...) {} } diff --git a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.cpp b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.cpp index 82c6a6a5b9b..5a975e7932c 100644 --- a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.cpp +++ b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.cpp @@ -12,14 +12,14 @@ CollapsingFinalBlockInputStream::~CollapsingFinalBlockInputStream() /// Нужно обезвредить все MergingBlockPtr, чтобы они не пытались класть блоки в output_blocks. previous.block.cancel(); last_positive.block.cancel(); - + while (!queue.empty()) { Cursor c = queue.top(); queue.pop(); c.block.cancel(); } - + for (size_t i = 0; i < output_blocks.size(); ++i) delete output_blocks[i]; } @@ -43,7 +43,7 @@ void CollapsingFinalBlockInputStream::fetchNextBlock(size_t input_index) Block block = stream->read(); if (!block) return; - MergingBlockPtr merging_block(new MergingBlock(block, input_index, description, sign_column, &output_blocks)); + MergingBlockPtr merging_block(new MergingBlock(block, input_index, description, sign_column_name, &output_blocks)); ++blocks_fetched; queue.push(Cursor(merging_block)); } @@ -56,18 +56,18 @@ void CollapsingFinalBlockInputStream::commitCurrent() { last_positive.addToFilter(); } - + if (!(count_positive == count_negative || count_positive + 1 == count_negative || count_positive == count_negative + 1)) { if (count_incorrect_data < MAX_ERROR_MESSAGES) reportBadCounts(); ++count_incorrect_data; } - + last_positive = Cursor(); previous = Cursor(); } - + count_negative = 0; count_positive = 0; } @@ -81,7 +81,7 @@ Block CollapsingFinalBlockInputStream::readImpl() first = false; } - + /// Будем формировать блоки для ответа, пока не получится непустой блок. while (true) { @@ -89,10 +89,10 @@ Block CollapsingFinalBlockInputStream::readImpl() { Cursor current = queue.top(); queue.pop(); - + bool has_next = !queue.empty(); Cursor next = has_next ? queue.top() : Cursor(); - + /// Будем продвигаться в текущем блоке, не используя очередь, пока возможно. while (true) { @@ -101,7 +101,7 @@ Block CollapsingFinalBlockInputStream::readImpl() commitCurrent(); previous = current; } - + Int8 sign = current.getSign(); if (sign == 1) { @@ -116,53 +116,50 @@ Block CollapsingFinalBlockInputStream::readImpl() } else reportBadSign(sign); - + if (current.isLast()) { fetchNextBlock(current.block->stream_index); - + /// Все потоки кончились. Обработаем последний ключ. if (!has_next) - { commitCurrent(); - } - + break; } else { current.next(); - + if (has_next && !(next < current)) { queue.push(current); - break; } } } } - + /// Конец потока. if (output_blocks.empty()) { if (blocks_fetched != blocks_output) LOG_ERROR(log, "Logical error: CollapsingFinalBlockInputStream has output " << blocks_output << " blocks instead of " << blocks_fetched); - + return Block(); } - + MergingBlock * merging_block = output_blocks.back(); Block block = merging_block->block; - + for (size_t i = 0; i < block.columns(); ++i) block.getByPosition(i).column = block.getByPosition(i).column->filter(merging_block->filter); - + output_blocks.pop_back(); delete merging_block; - + ++blocks_output; - + if (block) return block; } diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index fde2a68ce0a..1556403a8a3 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -31,7 +31,7 @@ Block IProfilingBlockInputStream::read() Block res; - if (is_cancelled) + if (is_cancelled.load(std::memory_order_seq_cst)) return res; res = readImpl(); @@ -226,7 +226,7 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) size_t rows_processed = process_list_elem->progress.rows; size_t bytes_processed = process_list_elem->progress.bytes; - size_t total_rows_estimate = std::max(process_list_elem->progress.rows, process_list_elem->progress.total_rows); + size_t total_rows_estimate = std::max(rows_processed, process_list_elem->progress.total_rows); /** Проверяем ограничения на объём данных для чтения, скорость выполнения запроса, квоту на объём данных для чтения. * NOTE: Может быть, имеет смысл сделать, чтобы они проверялись прямо в ProcessList? @@ -253,16 +253,32 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); } - if (limits.min_execution_speed) + size_t total_rows = process_list_elem->progress.total_rows; + + if (limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0)) { double total_elapsed = info.total_stopwatch.elapsedSeconds(); - if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0 - && rows_processed / total_elapsed < limits.min_execution_speed) + if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0) { - throw Exception("Query is executing too slow: " + toString(rows_processed / total_elapsed) - + " rows/sec., minimum: " + toString(limits.min_execution_speed), - ErrorCodes::TOO_SLOW); + if (limits.min_execution_speed && rows_processed / total_elapsed < limits.min_execution_speed) + throw Exception("Query is executing too slow: " + toString(rows_processed / total_elapsed) + + " rows/sec., minimum: " + toString(limits.min_execution_speed), + ErrorCodes::TOO_SLOW); + + size_t total_rows = process_list_elem->progress.total_rows; + + /// Если предсказанное время выполнения больше, чем max_execution_time. + if (limits.max_execution_time != 0 && total_rows) + { + double estimated_execution_time_seconds = total_elapsed * (static_cast(total_rows) / rows_processed); + + if (estimated_execution_time_seconds > limits.max_execution_time.totalSeconds()) + throw Exception("Estimated query execution time (" + toString(estimated_execution_time_seconds) + " seconds)" + + " is too long. Maximum: " + toString(limits.max_execution_time.totalSeconds()) + + ". Estimated rows to process: " + toString(total_rows), + ErrorCodes::TOO_SLOW); + } } } @@ -276,7 +292,8 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) void IProfilingBlockInputStream::cancel() { - if (!__sync_bool_compare_and_swap(&is_cancelled, false, true)) + bool old_val = false; + if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) return; for (auto & child : children) diff --git a/dbms/src/DataStreams/copyData.cpp b/dbms/src/DataStreams/copyData.cpp index 6d1b3197cd6..ecbb5e6ddbd 100644 --- a/dbms/src/DataStreams/copyData.cpp +++ b/dbms/src/DataStreams/copyData.cpp @@ -7,20 +7,30 @@ namespace DB { -void copyData(IBlockInputStream & from, IBlockOutputStream & to, volatile bool * is_cancelled) +namespace +{ + +bool isAtomicSet(std::atomic * val) +{ + return ((val != nullptr) && val->load(std::memory_order_seq_cst)); +} + +} + +void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic * is_cancelled) { from.readPrefix(); to.writePrefix(); while (Block block = from.read()) { - if (is_cancelled && *is_cancelled) + if (isAtomicSet(is_cancelled)) break; to.write(block); } - if (is_cancelled && *is_cancelled) + if (isAtomicSet(is_cancelled)) return; /// Для вывода дополнительной информации в некоторых форматах. @@ -33,7 +43,7 @@ void copyData(IBlockInputStream & from, IBlockOutputStream & to, volatile bool * to.setExtremes(input->getExtremes()); } - if (is_cancelled && *is_cancelled) + if (isAtomicSet(is_cancelled)) return; from.readSuffix(); diff --git a/dbms/src/DataStreams/tests/aggregating_stream.cpp b/dbms/src/DataStreams/tests/aggregating_stream.cpp index 5fd9cd26e97..323b34e7690 100644 --- a/dbms/src/DataStreams/tests/aggregating_stream.cpp +++ b/dbms/src/DataStreams/tests/aggregating_stream.cpp @@ -23,16 +23,18 @@ int main(int argc, char ** argv) { + using namespace DB; + try { size_t n = argc == 2 ? atoi(argv[1]) : 10; - DB::Block block; + Block block; - DB::ColumnWithNameAndType column_x; + ColumnWithNameAndType column_x; column_x.name = "x"; - column_x.type = new DB::DataTypeInt16; - DB::ColumnInt16 * x = new DB::ColumnInt16; + column_x.type = new DataTypeInt16; + ColumnInt16 * x = new ColumnInt16; column_x.column = x; auto & vec_x = x->getData(); @@ -44,65 +46,64 @@ int main(int argc, char ** argv) const char * strings[] = {"abc", "def", "abcd", "defg", "ac"}; - DB::ColumnWithNameAndType column_s1; + ColumnWithNameAndType column_s1; column_s1.name = "s1"; - column_s1.type = new DB::DataTypeString; - column_s1.column = new DB::ColumnString; + column_s1.type = new DataTypeString; + column_s1.column = new ColumnString; for (size_t i = 0; i < n; ++i) column_s1.column->insert(std::string(strings[i % 5])); block.insert(column_s1); - DB::ColumnWithNameAndType column_s2; + ColumnWithNameAndType column_s2; column_s2.name = "s2"; - column_s2.type = new DB::DataTypeString; - column_s2.column = new DB::ColumnString; + column_s2.type = new DataTypeString; + column_s2.column = new ColumnString; for (size_t i = 0; i < n; ++i) column_s2.column->insert(std::string(strings[i % 3])); block.insert(column_s2); - DB::ColumnNumbers key_column_numbers; - key_column_numbers.push_back(0); - //key_column_numbers.push_back(1); + Names key_column_names; + key_column_names.emplace_back("x"); - DB::AggregateFunctionFactory factory; + AggregateFunctionFactory factory; - DB::AggregateDescriptions aggregate_descriptions(1); + AggregateDescriptions aggregate_descriptions(1); - DB::DataTypes empty_list_of_types; + DataTypes empty_list_of_types; aggregate_descriptions[0].function = factory.get("count", empty_list_of_types); - Poco::SharedPtr result_types = new DB::DataTypes + Poco::SharedPtr result_types = new DataTypes { - new DB::DataTypeInt16, - // new DB::DataTypeString, - new DB::DataTypeUInt64, + new DataTypeInt16, + // new DataTypeString, + new DataTypeUInt64, }; - DB::Block sample; - for (DB::DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it) + Block sample; + for (DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it) { - DB::ColumnWithNameAndType col; + ColumnWithNameAndType col; col.type = *it; sample.insert(col); } - DB::BlockInputStreamPtr stream = new DB::OneBlockInputStream(block); - stream = new DB::AggregatingBlockInputStream(stream, key_column_numbers, aggregate_descriptions, false, true, - 0, DB::OverflowMode::THROW, nullptr, 0, 0); + BlockInputStreamPtr stream = new OneBlockInputStream(block); + stream = new AggregatingBlockInputStream(stream, key_column_names, aggregate_descriptions, false, true, + 0, OverflowMode::THROW, nullptr, 0, 0); - DB::WriteBufferFromOStream ob(std::cout); - DB::RowOutputStreamPtr row_out = new DB::TabSeparatedRowOutputStream(ob, sample); - DB::BlockOutputStreamPtr out = new DB::BlockOutputStreamFromRowOutputStream(row_out); + WriteBufferFromOStream ob(std::cout); + RowOutputStreamPtr row_out = new TabSeparatedRowOutputStream(ob, sample); + BlockOutputStreamPtr out = new BlockOutputStreamFromRowOutputStream(row_out); { Poco::Stopwatch stopwatch; stopwatch.start(); - DB::copyData(*stream, *out); + copyData(*stream, *out); stopwatch.stop(); std::cout << std::fixed << std::setprecision(2) @@ -115,7 +116,7 @@ int main(int argc, char ** argv) stream->dumpTree(std::cout); std::cout << std::endl; } - catch (const DB::Exception & e) + catch (const Exception & e) { std::cerr << e.displayText() << std::endl; } diff --git a/dbms/src/Functions/FunctionsDictionaries.cpp b/dbms/src/Functions/FunctionsDictionaries.cpp index 093bb0f532a..b5c29a04c88 100644 --- a/dbms/src/Functions/FunctionsDictionaries.cpp +++ b/dbms/src/Functions/FunctionsDictionaries.cpp @@ -31,6 +31,8 @@ void registerFunctionsDictionaries(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/src/IO/ReadBufferAIO.cpp b/dbms/src/IO/ReadBufferAIO.cpp new file mode 100644 index 00000000000..d47355bdfb2 --- /dev/null +++ b/dbms/src/IO/ReadBufferAIO.cpp @@ -0,0 +1,222 @@ +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, mode_t mode_, + char * existing_memory_) + : BufferWithOwnMemory(buffer_size_, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE), + fill_buffer(BufferWithOwnMemory(buffer_size_, nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)), + request_ptrs{ &request }, events(1), filename(filename_) +{ + ProfileEvents::increment(ProfileEvents::FileOpen); + + int open_flags = (flags_ == -1) ? O_RDONLY : flags_; + open_flags |= O_DIRECT; + + fd = ::open(filename.c_str(), open_flags, mode_); + if (fd == -1) + { + got_exception = true; + auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE; + throwFromErrno("Cannot open file " + filename, error_code); + } + + ::memset(&request, 0, sizeof(request)); +} + +ReadBufferAIO::~ReadBufferAIO() +{ + if (!got_exception) + { + try + { + waitForAIOCompletion(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + if (fd != -1) + ::close(fd); +} + +void ReadBufferAIO::setMaxBytes(size_t max_bytes_read_) +{ + if (is_started) + { + got_exception = true; + throw Exception("Illegal attempt to set the maximum number of bytes to read from file " + filename, ErrorCodes::LOGICAL_ERROR); + } + if ((max_bytes_read_ % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0) + { + got_exception = true; + throw Exception("Invalid maximum number of bytes to read from file " + filename, ErrorCodes::AIO_UNALIGNED_SIZE_ERROR); + } + max_bytes_read = max_bytes_read_; +} + +off_t ReadBufferAIO::seek(off_t off, int whence) +{ + if ((off % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0) + throw Exception("Invalid offset for ReadBufferAIO::seek", ErrorCodes::AIO_UNALIGNED_SIZE_ERROR); + + waitForAIOCompletion(); + + off_t new_pos; + + if (whence == SEEK_SET) + { + if (off < 0) + { + got_exception = true; + throw Exception("SEEK_SET underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + new_pos = off; + } + else if (whence == SEEK_CUR) + { + if (off >= 0) + { + if (off > (std::numeric_limits::max() - getPositionInFileRelaxed())) + { + got_exception = true; + throw Exception("SEEK_CUR overflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + } + else if (off < -getPositionInFileRelaxed()) + { + got_exception = true; + throw Exception("SEEK_CUR underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + new_pos = getPositionInFileRelaxed() + off; + } + else + { + got_exception = true; + throw Exception("ReadBufferAIO::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + + if (new_pos != getPositionInFileRelaxed()) + { + off_t working_buffer_begin_pos = pos_in_file - static_cast(working_buffer.size()); + if (hasPendingData() && (new_pos >= working_buffer_begin_pos) && (new_pos <= pos_in_file)) + { + /// Свдинулись, но остались в пределах буфера. + pos = working_buffer.begin() + (new_pos - working_buffer_begin_pos); + } + else + { + pos = working_buffer.end(); + pos_in_file = new_pos; + } + } + + return new_pos; +} + +off_t ReadBufferAIO::getPositionInFile() +{ + return seek(0, SEEK_CUR); +} + +off_t ReadBufferAIO::getPositionInFileRelaxed() const noexcept +{ + return pos_in_file - (working_buffer.end() - pos); +} + +bool ReadBufferAIO::nextImpl() +{ + /// Если конец файла уже был достигнут при вызове этой функции, + /// то текущий вызов ошибочен. + if (is_eof) + return false; + + waitForAIOCompletion(); + + /// При первом вызове не надо обменять местами основной и дублирующий буферы. + if (is_started) + swapBuffers(); + else + is_started = true; + + /// Если конец файла только что достигнут, больше ничего не делаем. + if (is_eof) + return true; + + /// Создать запрос. + request.aio_lio_opcode = IOCB_CMD_PREAD; + request.aio_fildes = fd; + request.aio_buf = reinterpret_cast(fill_buffer.internalBuffer().begin()); + request.aio_nbytes = std::min(fill_buffer.internalBuffer().size(), max_bytes_read); + request.aio_offset = pos_in_file; + request.aio_reqprio = 0; + + /// Отправить запрос. + while (io_submit(aio_context.ctx, request_ptrs.size(), &request_ptrs[0]) < 0) + if (errno != EINTR) + { + got_exception = true; + throw Exception("Cannot submit request for asynchronous IO on file " + filename, ErrorCodes::AIO_SUBMIT_ERROR); + } + + is_pending_read = true; + return true; +} + +void ReadBufferAIO::waitForAIOCompletion() +{ + if (is_pending_read) + { + while (io_getevents(aio_context.ctx, events.size(), events.size(), &events[0], nullptr) < 0) + if (errno != EINTR) + { + got_exception = true; + throw Exception("Failed to wait for asynchronous IO completion on file " + filename, ErrorCodes::AIO_COMPLETION_ERROR); + } + + is_pending_read = false; + off_t bytes_read = events[0].res; + + if (bytes_read < 0) + { + got_exception = true; + throw Exception("Asynchronous read error on file " + filename, ErrorCodes::AIO_READ_ERROR); + } + if ((bytes_read % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0) + { + got_exception = true; + throw Exception("Received unaligned number of bytes from file " + filename, ErrorCodes::AIO_UNALIGNED_SIZE_ERROR); + } + if (pos_in_file > (std::numeric_limits::max() - bytes_read)) + { + got_exception = true; + throw Exception("File position overflowed", ErrorCodes::LOGICAL_ERROR); + } + + pos_in_file += bytes_read; + total_bytes_read += bytes_read; + + if (bytes_read > 0) + fill_buffer.buffer().resize(bytes_read); + if ((static_cast(bytes_read) < fill_buffer.internalBuffer().size()) || (total_bytes_read == max_bytes_read)) + is_eof = true; + } +} + +void ReadBufferAIO::swapBuffers() noexcept +{ + internalBuffer().swap(fill_buffer.internalBuffer()); + buffer().swap(fill_buffer.buffer()); + std::swap(position(), fill_buffer.position()); +} + +} diff --git a/dbms/src/IO/WriteBufferAIO.cpp b/dbms/src/IO/WriteBufferAIO.cpp new file mode 100644 index 00000000000..7ac26c08b53 --- /dev/null +++ b/dbms/src/IO/WriteBufferAIO.cpp @@ -0,0 +1,202 @@ +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +WriteBufferAIO::WriteBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, mode_t mode_, + char * existing_memory_) + : BufferWithOwnMemory(buffer_size_, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE), + flush_buffer(BufferWithOwnMemory(buffer_size_, nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)), + request_ptrs{ &request }, events(1), filename(filename_) +{ + ProfileEvents::increment(ProfileEvents::FileOpen); + + int open_flags = (flags_ == -1) ? (O_WRONLY | O_TRUNC | O_CREAT) : flags_; + open_flags |= O_DIRECT; + + fd = ::open(filename.c_str(), open_flags, mode_); + if (fd == -1) + { + got_exception = true; + auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE; + throwFromErrno("Cannot open file " + filename, error_code); + } + + ::memset(&request, 0, sizeof(request)); +} + +WriteBufferAIO::~WriteBufferAIO() +{ + if (!got_exception) + { + try + { + flush(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + if (fd != -1) + ::close(fd); +} + +off_t WriteBufferAIO::seek(off_t off, int whence) +{ + if ((off % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0) + throw Exception("Invalid offset for WriteBufferAIO::seek", ErrorCodes::AIO_UNALIGNED_SIZE_ERROR); + + flush(); + + if (whence == SEEK_SET) + { + if (off < 0) + { + got_exception = true; + throw Exception("SEEK_SET underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + pos_in_file = off; + } + else if (whence == SEEK_CUR) + { + if (off >= 0) + { + if (off > (std::numeric_limits::max() - pos_in_file)) + { + got_exception = true; + throw Exception("SEEK_CUR overflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + } + else if (off < -pos_in_file) + { + got_exception = true; + throw Exception("SEEK_CUR underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + pos_in_file += off; + } + else + { + got_exception = true; + throw Exception("WriteBufferAIO::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + + return pos_in_file; +} + +off_t WriteBufferAIO::getPositionInFile() +{ + return seek(0, SEEK_CUR); +} + +void WriteBufferAIO::truncate(off_t length) +{ + if ((length % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0) + throw Exception("Invalid length for WriteBufferAIO::ftruncate", ErrorCodes::AIO_UNALIGNED_SIZE_ERROR); + + flush(); + + int res = ::ftruncate(fd, length); + if (res == -1) + { + got_exception = true; + throwFromErrno("Cannot truncate file " + filename, ErrorCodes::CANNOT_TRUNCATE_FILE); + } +} + +void WriteBufferAIO::sync() +{ + flush(); + + /// Попросим ОС сбросить данные на диск. + int res = ::fsync(fd); + if (res == -1) + { + got_exception = true; + throwFromErrno("Cannot fsync " + getFileName(), ErrorCodes::CANNOT_FSYNC); + } +} + +void WriteBufferAIO::flush() +{ + next(); + waitForAIOCompletion(); +} + +void WriteBufferAIO::nextImpl() +{ + if (!offset()) + return; + + waitForAIOCompletion(); + swapBuffers(); + + /// Создать запрос. + request.aio_lio_opcode = IOCB_CMD_PWRITE; + request.aio_fildes = fd; + request.aio_buf = reinterpret_cast(flush_buffer.buffer().begin()); + request.aio_nbytes = flush_buffer.offset(); + request.aio_offset = pos_in_file; + request.aio_reqprio = 0; + + if ((request.aio_nbytes % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0) + { + got_exception = true; + throw Exception("Illegal attempt to write unaligned data to file " + filename, ErrorCodes::AIO_UNALIGNED_SIZE_ERROR); + } + + /// Отправить запрос. + while (io_submit(aio_context.ctx, request_ptrs.size(), &request_ptrs[0]) < 0) + if (errno != EINTR) + { + got_exception = true; + throw Exception("Cannot submit request for asynchronous IO on file " + filename, ErrorCodes::AIO_SUBMIT_ERROR); + } + + is_pending_write = true; +} + +void WriteBufferAIO::waitForAIOCompletion() +{ + if (is_pending_write) + { + while (io_getevents(aio_context.ctx, events.size(), events.size(), &events[0], nullptr) < 0) + if (errno != EINTR) + { + got_exception = true; + throw Exception("Failed to wait for asynchronous IO completion on file " + filename, ErrorCodes::AIO_COMPLETION_ERROR); + } + + is_pending_write = false; + off_t bytes_written = events[0].res; + + if ((bytes_written < 0) || (static_cast(bytes_written) < flush_buffer.offset())) + { + got_exception = true; + throw Exception("Asynchronous write error on file " + filename, ErrorCodes::AIO_WRITE_ERROR); + } + if (pos_in_file > (std::numeric_limits::max() - bytes_written)) + { + got_exception = true; + throw Exception("File position overflowed", ErrorCodes::LOGICAL_ERROR); + } + + pos_in_file += bytes_written; + } +} + +void WriteBufferAIO::swapBuffers() noexcept +{ + buffer().swap(flush_buffer.buffer()); + std::swap(position(), flush_buffer.position()); +} + +} diff --git a/dbms/src/IO/tests/read_buffer_aio.cpp b/dbms/src/IO/tests/read_buffer_aio.cpp new file mode 100644 index 00000000000..3dae95e9f7e --- /dev/null +++ b/dbms/src/IO/tests/read_buffer_aio.cpp @@ -0,0 +1,337 @@ +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + +namespace +{ + +void run(); +void prepare(std::string & directory, std::string & filename, std::string & buf); +void die(const std::string & msg); +void run_test(unsigned int num, const std::function func); + +bool test1(const std::string & filename); +bool test2(const std::string & filename, const std::string & buf); +bool test3(const std::string & filename, const std::string & buf); +bool test4(const std::string & filename, const std::string & buf); +bool test5(const std::string & filename); +bool test6(const std::string & filename, const std::string & buf); +bool test7(const std::string & filename, const std::string & buf); +bool test8(const std::string & filename); +bool test9(const std::string & filename, const std::string & buf); +bool test10(const std::string & filename, const std::string & buf); +bool test11(const std::string & filename); +bool test12(const std::string & filename, const std::string & buf); + +void run() +{ + namespace fs = boost::filesystem; + + std::string directory; + std::string filename; + std::string buf; + prepare(directory, filename, buf); + + const std::vector > tests = + { + std::bind(test1, std::ref(filename)), + std::bind(test2, std::ref(filename), std::ref(buf)), + std::bind(test3, std::ref(filename), std::ref(buf)), + std::bind(test4, std::ref(filename), std::ref(buf)), + std::bind(test5, std::ref(filename)), + std::bind(test6, std::ref(filename), std::ref(buf)), + std::bind(test7, std::ref(filename), std::ref(buf)), + std::bind(test8, std::ref(filename)), + std::bind(test9, std::ref(filename), std::ref(buf)), + std::bind(test10, std::ref(filename), std::ref(buf)), + std::bind(test11, std::ref(filename)), + std::bind(test12, std::ref(filename), std::ref(buf)) + }; + + unsigned int num = 0; + for (const auto & test : tests) + { + ++num; + run_test(num, test); + } + + fs::remove_all(directory); +} + +void prepare(std::string & directory, std::string & filename, std::string & buf) +{ + static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + + char pattern[] = "/tmp/fileXXXXXX"; + char * dir = ::mkdtemp(pattern); + if (dir == nullptr) + die("Could not create directory"); + + directory = std::string(dir); + filename = directory + "/foo"; + + size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; + buf.reserve(n); + + for (size_t i = 0; i < n; ++i) + buf += symbols[i % symbols.length()]; + + std::ofstream out(filename.c_str()); + if (!out.is_open()) + die("Could not open file"); + + out << buf; +} + +void die(const std::string & msg) +{ + std::cout << msg << "\n"; + ::exit(EXIT_FAILURE); +} + +void run_test(unsigned int num, const std::function func) +{ + bool ok; + + try + { + ok = func(); + } + catch (const DB::Exception & ex) + { + ok = false; + std::cout << "Caught exception " << ex.displayText() << "\n"; + } + catch (const std::exception & ex) + { + ok = false; + std::cout << "Caught exception " << ex.what() << "\n"; + } + + if (ok) + std::cout << "Test " << num << " passed\n"; + else + std::cout << "Test " << num << " failed\n"; +} + +bool test1(const std::string & filename) +{ + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + if (in.getFileName() != filename) + return false; + if (in.getFD() == -1) + return false; + return true; +} + +bool test2(const std::string & filename, const std::string & buf) +{ + std::string newbuf; + newbuf.resize(buf.length()); + + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + size_t count = in.read(&newbuf[0], newbuf.length()); + if (count != newbuf.length()) + return false; + + return (newbuf == buf); +} + +bool test3(const std::string & filename, const std::string & buf) +{ + std::string newbuf; + newbuf.resize(buf.length()); + + size_t requested = 9 * DEFAULT_AIO_FILE_BLOCK_SIZE; + + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + in.setMaxBytes(requested); + size_t count = in.read(&newbuf[0], newbuf.length()); + + newbuf.resize(count); + return (newbuf == buf.substr(0, requested)); +} + +bool test4(const std::string & filename, const std::string & buf) +{ + std::string newbuf; + newbuf.resize(buf.length()); + + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + in.setMaxBytes(0); + size_t n_read = in.read(&newbuf[0], newbuf.length()); + + return n_read == 0; +} + +bool test5(const std::string & filename) +{ + bool ok = false; + + try + { + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + in.setMaxBytes(DEFAULT_AIO_FILE_BLOCK_SIZE >> 1); + } + catch (const DB::Exception &) + { + ok = true; + } + return ok; +} + +bool test6(const std::string & filename, const std::string & buf) +{ + std::string newbuf; + newbuf.resize(buf.length()); + + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + + if (in.getPositionInFile() != 0) + return false; + + size_t count = in.read(&newbuf[0], newbuf.length()); + if (count != newbuf.length()) + return false; + + if (static_cast(in.getPositionInFile()) != buf.length()) + return false; + + return true; +} + +bool test7(const std::string & filename, const std::string & buf) +{ + std::string newbuf; + newbuf.resize(buf.length() - DEFAULT_AIO_FILE_BLOCK_SIZE); + + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + (void) in.seek(DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_SET); + size_t count = in.read(&newbuf[0], newbuf.length()); + if (count != (9 * DEFAULT_AIO_FILE_BLOCK_SIZE)) + return false; + + return (newbuf == buf.substr(DEFAULT_AIO_FILE_BLOCK_SIZE)); +} + +bool test8(const std::string & filename) +{ + bool ok = false; + + try + { + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + (void) in.seek(DEFAULT_AIO_FILE_BLOCK_SIZE + 1, SEEK_CUR); + } + catch (const DB::Exception &) + { + ok = true; + } + + return ok; +} + +bool test9(const std::string & filename, const std::string & buf) +{ + bool ok = false; + + try + { + std::string newbuf; + newbuf.resize(buf.length()); + + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + size_t count = in.read(&newbuf[0], newbuf.length()); + if (count != newbuf.length()) + return false; + in.setMaxBytes(9 * DEFAULT_AIO_FILE_BLOCK_SIZE); + } + catch (const DB::Exception &) + { + ok = true; + } + + return ok; +} + +bool test10(const std::string & filename, const std::string & buf) +{ + std::string newbuf; + newbuf.resize(4 * DEFAULT_AIO_FILE_BLOCK_SIZE); + + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + size_t count1 = in.read(&newbuf[0], newbuf.length()); + if (count1 != newbuf.length()) + return false; + + if (newbuf != buf.substr(0, 4 * DEFAULT_AIO_FILE_BLOCK_SIZE)) + return false; + + (void) in.seek(2 * DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_CUR); + + size_t count2 = in.read(&newbuf[0], newbuf.length()); + if (count2 != newbuf.length()) + return false; + + if (newbuf != buf.substr(6 * DEFAULT_AIO_FILE_BLOCK_SIZE)) + return false; + + return true; +} + +bool test11(const std::string & filename) +{ + bool ok = false; + + try + { + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + (void) in.seek(-DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_SET); + } + catch (const DB::Exception &) + { + ok = true; + } + + return ok; +} + +bool test12(const std::string & filename, const std::string & buf) +{ + bool ok = false; + + try + { + std::string newbuf; + newbuf.resize(4 * DEFAULT_AIO_FILE_BLOCK_SIZE); + + DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + size_t count = in.read(&newbuf[0], newbuf.length()); + if (count != newbuf.length()) + return false; + + (void) in.seek(-(10 * DEFAULT_AIO_FILE_BLOCK_SIZE), SEEK_CUR); + } + catch (const DB::Exception &) + { + ok = true; + } + + return ok; +} + +} + +int main() +{ + run(); + return 0; +} diff --git a/dbms/src/IO/tests/write_buffer_aio.cpp b/dbms/src/IO/tests/write_buffer_aio.cpp new file mode 100644 index 00000000000..3dbdbd0e522 --- /dev/null +++ b/dbms/src/IO/tests/write_buffer_aio.cpp @@ -0,0 +1,291 @@ +#include +#include + +#include + +#include +#include +#include +#include + +namespace +{ + +void run(); +void die(const std::string & msg); +void run_test(unsigned int num, const std::function func); + +bool test1(); +bool test2(); +bool test3(); +bool test4(); + +void run() +{ + const std::vector > tests = + { + test1, + test2, + test3, + test4 + }; + + unsigned int num = 0; + for (const auto & test : tests) + { + ++num; + run_test(num, test); + } +} + +void die(const std::string & msg) +{ + std::cout << msg; + ::exit(EXIT_FAILURE); +} + +void run_test(unsigned int num, const std::function func) +{ + bool ok; + + try + { + ok = func(); + } + catch (const DB::Exception & ex) + { + ok = false; + std::cout << "Caught exception " << ex.displayText() << "\n"; + } + catch (const std::exception & ex) + { + ok = false; + std::cout << "Caught exception " << ex.what() << "\n"; + } + + if (ok) + std::cout << "Test " << num << " passed\n"; + else + std::cout << "Test " << num << " failed\n"; +} + +bool test1() +{ + namespace fs = boost::filesystem; + + static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + + char pattern[] = "/tmp/fileXXXXXX"; + char * dir = ::mkdtemp(pattern); + if (dir == nullptr) + die("Could not create directory"); + + const std::string directory = std::string(dir); + const std::string filename = directory + "/foo"; + + size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; + + std::string buf; + buf.reserve(n); + + for (size_t i = 0; i < n; ++i) + buf += symbols[i % symbols.length()]; + + { + DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + + if (out.getFileName() != filename) + return false; + if (out.getFD() == -1) + return false; + + out.write(&buf[0], buf.length()); + } + + std::ifstream in(filename.c_str()); + if (!in.is_open()) + die("Could not open file"); + + std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; + + in.close(); + fs::remove_all(directory); + + return (received == buf); +} + +bool test2() +{ + namespace fs = boost::filesystem; + + static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + + char pattern[] = "/tmp/fileXXXXXX"; + char * dir = ::mkdtemp(pattern); + if (dir == nullptr) + die("Could not create directory"); + + const std::string directory = std::string(dir); + const std::string filename = directory + "/foo"; + + size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; + + std::string buf; + buf.reserve(n); + + for (size_t i = 0; i < n; ++i) + buf += symbols[i % symbols.length()]; + + { + DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + + if (out.getFileName() != filename) + return false; + if (out.getFD() == -1) + return false; + + out.write(&buf[0], buf.length() / 2); + out.seek(DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_CUR); + out.write(&buf[buf.length() / 2], buf.length() / 2); + } + + std::ifstream in(filename.c_str()); + if (!in.is_open()) + die("Could not open file"); + + std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; + + in.close(); + fs::remove_all(directory); + + if (received.substr(0, buf.length() / 2) != buf.substr(0, buf.length() / 2)) + return false; + if (received.substr(buf.length() / 2, DEFAULT_AIO_FILE_BLOCK_SIZE) != std::string(DEFAULT_AIO_FILE_BLOCK_SIZE, '\0')) + return false; + if (received.substr(buf.length() / 2 + DEFAULT_AIO_FILE_BLOCK_SIZE) != buf.substr(buf.length() / 2)) + return false; + + return true; +} + +bool test3() +{ + namespace fs = boost::filesystem; + + static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + + char pattern[] = "/tmp/fileXXXXXX"; + char * dir = ::mkdtemp(pattern); + if (dir == nullptr) + die("Could not create directory"); + + const std::string directory = std::string(dir); + const std::string filename = directory + "/foo"; + + size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; + + std::string buf; + buf.reserve(n); + + for (size_t i = 0; i < n; ++i) + buf += symbols[i % symbols.length()]; + + { + DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + + if (out.getFileName() != filename) + return false; + if (out.getFD() == -1) + return false; + + out.write(&buf[0], buf.length()); + + off_t pos1 = out.getPositionInFile(); + + out.truncate(buf.length() / 2); + + off_t pos2 = out.getPositionInFile(); + + if (pos1 != pos2) + return false; + } + + std::ifstream in(filename.c_str()); + if (!in.is_open()) + die("Could not open file"); + + std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; + + in.close(); + fs::remove_all(directory); + + return (received == buf.substr(0, buf.length() / 2)); +} + +bool test4() +{ + namespace fs = boost::filesystem; + + static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + + char pattern[] = "/tmp/fileXXXXXX"; + char * dir = ::mkdtemp(pattern); + if (dir == nullptr) + die("Could not create directory"); + + const std::string directory = std::string(dir); + const std::string filename = directory + "/foo"; + + size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE; + + std::string buf; + buf.reserve(n); + + for (size_t i = 0; i < n; ++i) + buf += symbols[i % symbols.length()]; + + { + DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); + + if (out.getFileName() != filename) + return false; + if (out.getFD() == -1) + return false; + + out.write(&buf[0], buf.length()); + + off_t pos1 = out.getPositionInFile(); + + out.truncate(3 * buf.length() / 2); + + off_t pos2 = out.getPositionInFile(); + + if (pos1 != pos2) + return false; + } + + std::ifstream in(filename.c_str()); + if (!in.is_open()) + die("Could not open file"); + + std::string received{ std::istreambuf_iterator(in), std::istreambuf_iterator() }; + + in.close(); + fs::remove_all(directory); + + if (received.substr(0, buf.length()) != buf) + return false; + + if (received.substr(buf.length()) != std::string(buf.length() / 2, '\0')) + return false; + + return true; +} + +} + +int main() +{ + run(); + return 0; +} diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index ee21f49c893..9a9522361aa 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -482,12 +482,13 @@ void NO_INLINE Aggregator::executeImplCase( /// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом. if (inserted) { - method.onNewKey(*it, keys_size, i, keys, *aggregates_pool); - AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); /// exception-safety - если не удалось выделить память или создать состояния, то не будут вызываться деструкторы. aggregate_data = nullptr; + + method.onNewKey(*it, keys_size, i, keys, *aggregates_pool); + AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states); createAggregateStates(place); aggregate_data = place; @@ -981,20 +982,30 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( /// packaged_task используются, чтобы исключения автоматически прокидывались в основной поток. - std::vector> tasks; - tasks.reserve(Method::Data::NUM_BUCKETS); + std::vector> tasks(Method::Data::NUM_BUCKETS); - for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket) + try { - if (method.data.impls[bucket].empty()) - continue; + for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket) + { + if (method.data.impls[bucket].empty()) + continue; - tasks.emplace_back(std::bind(converter, bucket, current_memory_tracker)); + tasks[bucket] = std::packaged_task(std::bind(converter, bucket, current_memory_tracker)); + if (thread_pool) + thread_pool->schedule([bucket, &tasks] { tasks[bucket](); }); + else + tasks[bucket](); + } + } + catch (...) + { + /// Если этого не делать, то в случае исключения, tasks уничтожится раньше завершения потоков, и будет плохо. if (thread_pool) - thread_pool->schedule([bucket, &tasks] { tasks[bucket](); }); - else - tasks[bucket](); + thread_pool->wait(); + + throw; } if (thread_pool) @@ -1008,6 +1019,9 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( std::exception_ptr first_exception; for (auto & task : tasks) { + if (!task.valid()) + continue; + try { blocks.emplace_back(task.get_future().get()); @@ -1054,15 +1068,35 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b && data_variants.isTwoLevel()) /// TODO Использовать общий тред-пул с функцией merge. thread_pool.reset(new boost::threadpool::pool(max_threads)); - if (data_variants.type == AggregatedDataVariants::Type::without_key || overflow_row) - blocks.splice(blocks.end(), prepareBlocksAndFillWithoutKey(data_variants, final)); - - if (data_variants.type != AggregatedDataVariants::Type::without_key) + try { - if (!data_variants.isTwoLevel()) - blocks.splice(blocks.end(), prepareBlocksAndFillSingleLevel(data_variants, final)); - else - blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get())); + if (data_variants.type == AggregatedDataVariants::Type::without_key || overflow_row) + blocks.splice(blocks.end(), prepareBlocksAndFillWithoutKey(data_variants, final)); + + if (data_variants.type != AggregatedDataVariants::Type::without_key) + { + if (!data_variants.isTwoLevel()) + blocks.splice(blocks.end(), prepareBlocksAndFillSingleLevel(data_variants, final)); + else + blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get())); + } + } + catch (...) + { + /** Если был хотя бы один эксепшен, то следует "откатить" владение состояниями агрегатных функций в ColumnAggregateFunction-ах + * - то есть, очистить их (см. комментарий в функции prepareBlockAndFill.) + */ + for (auto & block : blocks) + { + for (size_t column_num = keys_size; column_num < keys_size + aggregates_size; ++column_num) + { + IColumn & col = *block.getByPosition(column_num).column; + if (ColumnAggregateFunction * col_aggregate = typeid_cast(&col)) + col_aggregate->getData().clear(); + } + } + + throw; } if (!final) @@ -1112,13 +1146,13 @@ void NO_INLINE Aggregator::mergeDataImpl( for (size_t i = 0; i < aggregates_size; ++i) aggregate_functions[i]->destroy( Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); - - Method::getAggregateData(it->second) = nullptr; } else { res_it->second = it->second; } + + Method::getAggregateData(it->second) = nullptr; } } @@ -1194,24 +1228,35 @@ void NO_INLINE Aggregator::mergeTwoLevelDataImpl( /// packaged_task используются, чтобы исключения автоматически прокидывались в основной поток. - std::vector> tasks; - tasks.reserve(Method::Data::NUM_BUCKETS); + std::vector> tasks(Method::Data::NUM_BUCKETS); - for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket) + try { - tasks.emplace_back(std::bind(merge_bucket, bucket, current_memory_tracker)); + for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket) + { + tasks[bucket] = std::packaged_task(std::bind(merge_bucket, bucket, current_memory_tracker)); + if (thread_pool) + thread_pool->schedule([bucket, &tasks] { tasks[bucket](); }); + else + tasks[bucket](); + } + } + catch (...) + { + /// Если этого не делать, то в случае исключения, tasks уничтожится раньше завершения потоков, и будет плохо. if (thread_pool) - thread_pool->schedule([bucket, &tasks] { tasks[bucket](); }); - else - tasks[bucket](); + thread_pool->wait(); + + throw; } if (thread_pool) thread_pool->wait(); for (auto & task : tasks) - task.get_future().get(); + if (task.valid()) + task.get_future().get(); } @@ -1333,7 +1378,7 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va template void NO_INLINE Aggregator::mergeStreamsImpl( Block & block, - AggregatedDataVariants & result, + const Sizes & key_sizes, Arena * aggregates_pool, Method & method, Table & data) const @@ -1360,17 +1405,17 @@ void NO_INLINE Aggregator::mergeStreamsImpl( bool inserted; /// Вставили новый ключ, или такой ключ уже был? /// Получаем ключ для вставки в хэш-таблицу. - auto key = state.getKey(key_columns, keys_size, i, result.key_sizes, keys, *aggregates_pool); + auto key = state.getKey(key_columns, keys_size, i, key_sizes, keys, *aggregates_pool); data.emplace(key, it, inserted); if (inserted) { + AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); + aggregate_data = nullptr; + method.onNewKey(*it, keys_size, i, keys, *aggregates_pool); - AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); - - aggregate_data = nullptr; AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states); createAggregateStates(place); aggregate_data = place; @@ -1426,9 +1471,6 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants Block empty_block; initialize(empty_block); - /// result будет уничтожать состояния агрегатных функций в деструкторе - result.aggregator = this; - /** Если на удалённых серверах использовался двухуровневый метод агрегации, * то в блоках будет расположена информация о номере корзины. * Тогда вычисления можно будет распараллелить по корзинам. @@ -1468,7 +1510,8 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants * - в случае одноуровневой агрегации, а также для блоков с "переполнившимися" значениями. * Если есть хотя бы один блок с номером корзины больше нуля, значит была двухуровневая агрегация. */ - size_t has_two_level = bucket_to_blocks.rbegin()->first > 0; + UInt32 max_bucket = bucket_to_blocks.rbegin()->first; + size_t has_two_level = max_bucket > 0; if (has_two_level) { @@ -1481,6 +1524,9 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants #undef M } + /// result будет уничтожать состояния агрегатных функций в деструкторе + result.aggregator = this; + result.init(method); result.keys_size = keys_size; result.key_sizes = key_sizes; @@ -1492,12 +1538,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants { LOG_TRACE(log, "Merging partially aggregated two-level data."); - std::unique_ptr thread_pool; - if (max_threads > 1 && total_input_rows > 100000 /// TODO Сделать настраиваемый порог. - && has_two_level) - thread_pool.reset(new boost::threadpool::pool(max_threads)); - - auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker) + auto merge_bucket = [&bucket_to_blocks, &result, &key_sizes, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker) { current_memory_tracker = memory_tracker; @@ -1505,7 +1546,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants { #define M(NAME) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, result, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket]); + mergeStreamsImpl(block, key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket]); if (false) {} APPLY_FOR_VARIANTS_TWO_LEVEL(M) @@ -1517,12 +1558,16 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants /// packaged_task используются, чтобы исключения автоматически прокидывались в основной поток. - std::vector> tasks; - tasks.reserve(bucket_to_blocks.size() - has_blocks_with_unknown_bucket); + std::vector> tasks(max_bucket + 1); - for (auto & bucket_blocks : bucket_to_blocks) + std::unique_ptr thread_pool; + if (max_threads > 1 && total_input_rows > 100000 /// TODO Сделать настраиваемый порог. + && has_two_level) + thread_pool.reset(new boost::threadpool::pool(max_threads)); + + for (const auto & bucket_blocks : bucket_to_blocks) { - auto bucket = bucket_blocks.first; + const auto bucket = bucket_blocks.first; if (bucket == -1) continue; @@ -1530,7 +1575,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants result.aggregates_pools.push_back(new Arena); Arena * aggregates_pool = result.aggregates_pools.back().get(); - tasks.emplace_back(std::bind(merge_bucket, bucket, aggregates_pool, current_memory_tracker)); + tasks[bucket] = std::packaged_task(std::bind(merge_bucket, bucket, aggregates_pool, current_memory_tracker)); if (thread_pool) thread_pool->schedule([bucket, &tasks] { tasks[bucket](); }); @@ -1542,7 +1587,8 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants thread_pool->wait(); for (auto & task : tasks) - task.get_future().get(); + if (task.valid()) + task.get_future().get(); LOG_TRACE(log, "Merged partially aggregated two-level data."); } @@ -1559,7 +1605,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, result, result.aggregates_pool, *result.NAME, result.NAME->data); + mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data); APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M diff --git a/dbms/src/Interpreters/DictionaryFactory.cpp b/dbms/src/Interpreters/DictionaryFactory.cpp index d36926a0c29..38d973da08f 100644 --- a/dbms/src/Interpreters/DictionaryFactory.cpp +++ b/dbms/src/Interpreters/DictionaryFactory.cpp @@ -41,10 +41,10 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab } else if ("cache" == layout_type) { - const auto size = config.getInt(layout_prefix + ".cache.size"); + const auto size = config.getInt(layout_prefix + ".cache.size_in_cells"); if (size == 0) throw Exception{ - "Dictionary of type 'cache' cannot have size of 0 bytes", + "Dictionary of type 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE }; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 237008d2bd0..ed20ee30054 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -85,6 +85,9 @@ void ExpressionAnalyzer::init() /// GROUP BY injective function elimination. optimizeGroupBy(); + /// Удалить из ORDER BY повторяющиеся элементы. + optimizeOrderBy(); + /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(); @@ -162,11 +165,13 @@ void ExpressionAnalyzer::analyzeAggregation() } NameAndTypePair key{column_name, col.type}; - aggregation_keys.push_back(key); + /// Ключи агрегации уникализируются. if (!unique_keys.count(key.name)) { unique_keys.insert(key.name); + aggregation_keys.push_back(key); + /// key is no longer needed, therefore we can save a little by moving it aggregated_columns.push_back(std::move(key)); } @@ -529,6 +534,38 @@ void ExpressionAnalyzer::optimizeGroupBy() } +void ExpressionAnalyzer::optimizeOrderBy() +{ + if (!(select_query && select_query->order_expression_list)) + return; + + /// Уникализируем условия сортировки. + using NameAndLocale = std::pair; + std::set elems_set; + + ASTs & elems = select_query->order_expression_list->children; + ASTs unique_elems; + unique_elems.reserve(elems.size()); + + for (const auto & elem : elems) + { + String name = elem->children.front()->getColumnName(); + const ASTOrderByElement & order_by_elem = typeid_cast(*elem); + + if (elems_set.emplace( + std::piecewise_construct, + std::forward_as_tuple(name), + std::forward_as_tuple(order_by_elem.collator ? order_by_elem.collator->getLocale() : std::string())).second) + { + unique_elems.emplace_back(elem); + } + } + + if (unique_elems.size() < elems.size()) + elems = unique_elems; +} + + void ExpressionAnalyzer::makeSetsForIndex() { if (storage && ast && storage->supportsIndexForIn()) @@ -1306,6 +1343,7 @@ void ExpressionAnalyzer::getAggregates(ASTPtr ast, ExpressionActionsPtr & action AggregateDescription aggregate; aggregate.column_name = node->getColumnName(); + /// Агрегатные функции уникализируются. for (size_t i = 0; i < aggregate_descriptions.size(); ++i) if (aggregate_descriptions[i].column_name == aggregate.column_name) return; @@ -1711,7 +1749,8 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions() void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) { for (NamesAndTypesList::iterator it = aggregation_keys.begin(); it != aggregation_keys.end(); ++it) - key_names.push_back(it->name); + key_names.emplace_back(it->name); + aggregates = aggregate_descriptions; } diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index f7ce06720ac..922ad484e0b 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -4,136 +4,49 @@ #include #include #include +#include namespace DB { namespace { - std::string getDictionariesConfigPath(const Poco::Util::AbstractConfiguration & config) + std::set getDictionariesConfigPaths(const Poco::Util::AbstractConfiguration & config) { - const auto path = config.getString("dictionaries_config", ""); - if (path.empty()) - return path; + auto pattern = config.getString("dictionaries_config", ""); + if (pattern.empty()) + return {}; - if (path[0] != '/') + std::set files; + if (pattern[0] != '/') { const auto app_config_path = config.getString("config-file", "config.xml"); const auto config_dir = Poco::Path{app_config_path}.parent().toString(); - const auto absolute_path = config_dir + path; - if (Poco::File{absolute_path}.exists()) - return absolute_path; + const auto absolute_path = config_dir + pattern; + Poco::Glob::glob(absolute_path, files, 0); + if (!files.empty()) + return files; } - return path; + Poco::Glob::glob(pattern, files, 0); + + return files; } } void ExternalDictionaries::reloadImpl() { - const auto config_path = getDictionariesConfigPath(Poco::Util::Application::instance().config()); - const Poco::File config_file{config_path}; + const auto config_paths = getDictionariesConfigPaths(Poco::Util::Application::instance().config()); - if (config_path.empty() || !config_file.exists()) - { - LOG_WARNING(log, "config file '" + config_path + "' does not exist"); - } - else - { - const auto last_modified = config_file.getLastModified(); - if (last_modified > config_last_modified) - { - stored_exceptions.clear(); - - /// definitions of dictionaries may have changed, recreate all of them - config_last_modified = last_modified; - - const auto config = new Poco::Util::XMLConfiguration{config_path}; - SCOPE_EXIT( - config->release(); - ); - - /// get all dictionaries' definitions - Poco::Util::AbstractConfiguration::Keys keys; - config->keys(keys); - - /// for each dictionary defined in xml config - for (const auto & key : keys) - { - std::string name; - - try - { - if (0 != strncmp(key.data(), "dictionary", strlen("dictionary"))) - { - LOG_WARNING(log, "unknown node in dictionaries file: '" + key + "', 'dictionary'"); - continue; - } - - name = config->getString(key + ".name"); - if (name.empty()) - { - LOG_WARNING(log, "dictionary name cannot be empty"); - continue; - } - - auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context); - if (!dict_ptr->isCached()) - { - const auto & lifetime = dict_ptr->getLifetime(); - if (lifetime.min_sec != 0 && lifetime.max_sec != 0) - { - std::uniform_int_distribution distribution{ - lifetime.min_sec, - lifetime.max_sec - }; - update_times[name] = std::chrono::system_clock::now() + - std::chrono::seconds{distribution(rnd_engine)}; - } - } - - auto it = dictionaries.find(name); - /// add new dictionary or update an existing version - if (it == std::end(dictionaries)) - { - const std::lock_guard lock{dictionaries_mutex}; - dictionaries.emplace(name, std::make_shared>(dict_ptr.release())); - } - else - it->second->set(dict_ptr.release()); - } - catch (...) - { - if (!name.empty()) - stored_exceptions.emplace(name, std::current_exception()); - - try - { - throw; - } - catch (const Poco::Exception & e) - { - LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually. " << e.displayText()); - } - catch (const std::exception & e) - { - LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually. " << e.what()); - } - catch (...) - { - LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually."); - } - } - } - } - } + for (const auto & config_path : config_paths) + reloadFromFile(config_path); /// periodic update for (auto & dictionary : dictionaries) { try { - auto current = dictionary.second->get(); + auto current = dictionary.second.first->get(); const auto & lifetime = current->getLifetime(); /// do not update dictionaries with zero as lifetime @@ -160,7 +73,7 @@ void ExternalDictionaries::reloadImpl() { /// create new version of dictionary auto new_version = current->clone(); - dictionary.second->set(new_version.release()); + dictionary.second.first->set(new_version.release()); } } } @@ -171,7 +84,7 @@ void ExternalDictionaries::reloadImpl() } catch (const std::exception & e) { - LOG_ERROR(log, "Cannot load external dictionary '" << dictionary.first + LOG_ERROR(log, "Cannot update external dictionary '" << dictionary.first << "'! You must resolve this manually. " << e.what()); } catch (...) @@ -182,4 +95,116 @@ void ExternalDictionaries::reloadImpl() } } +void ExternalDictionaries::reloadFromFile(const std::string & config_path) +{ + const Poco::File config_file{config_path}; + + if (config_path.empty() || !config_file.exists()) + { + LOG_WARNING(log, "config file '" + config_path + "' does not exist"); + } + else + { + auto it = last_modification_times.find(config_path); + if (it == std::end(last_modification_times)) + it = last_modification_times.emplace(config_path, Poco::Timestamp{0}).first; + auto & config_last_modified = it->second; + + const auto last_modified = config_file.getLastModified(); + if (last_modified > config_last_modified) + { + stored_exceptions.clear(); + + /// definitions of dictionaries may have changed, recreate all of them + config_last_modified = last_modified; + + const auto config = new Poco::Util::XMLConfiguration{config_path}; + SCOPE_EXIT(config->release()); + + /// get all dictionaries' definitions + Poco::Util::AbstractConfiguration::Keys keys; + config->keys(keys); + + /// for each dictionary defined in xml config + for (const auto & key : keys) + { + std::string name; + + if (0 != strncmp(key.data(), "dictionary", strlen("dictionary"))) + { + LOG_WARNING(log, config_path << ": unknown node in dictionaries file: '" << key + "', 'dictionary'"); + continue; + } + + try + { + name = config->getString(key + ".name"); + if (name.empty()) + { + LOG_WARNING(log, config_path << ": dictionary name cannot be empty"); + continue; + } + + auto it = dictionaries.find(name); + if (it != std::end(dictionaries)) + if (it->second.second != config_path) + throw std::runtime_error{"Overriding dictionary from file " + it->second.second}; + + auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context); + if (!dict_ptr->isCached()) + { + const auto & lifetime = dict_ptr->getLifetime(); + if (lifetime.min_sec != 0 && lifetime.max_sec != 0) + { + std::uniform_int_distribution distribution{ + lifetime.min_sec, + lifetime.max_sec + }; + update_times[name] = std::chrono::system_clock::now() + + std::chrono::seconds{distribution(rnd_engine)}; + } + } + + /// add new dictionary or update an existing version + if (it == std::end(dictionaries)) + { + const std::lock_guard lock{dictionaries_mutex}; + dictionaries.emplace(name, dictionary_origin_pair_t{ + std::make_shared>(dict_ptr.release()), + config_path + }); + } + else + it->second.first->set(dict_ptr.release()); + } + catch (...) + { + if (!name.empty()) + stored_exceptions.emplace(name, std::current_exception()); + + try + { + throw; + } + catch (const Poco::Exception & e) + { + LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name + << "'! You must resolve this manually. " << e.displayText()); + } + catch (const std::exception & e) + { + LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name + << "'! You must resolve this manually. " << e.what()); + } + catch (...) + { + LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name + << "'! You must resolve this manually."); + } + } + } + } + } +} + } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 2c676e2b3c1..13e1779243a 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -664,8 +664,15 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu * то в качестве размера блока будем использовать limit + offset (чтобы не читать из таблицы больше, чем запрошено), * а также установим количество потоков в 1 и отменим асинхронное выполнение конвейера запроса. */ - if (!query.distinct && !query.prewhere_expression && !query.where_expression && !query.group_expression_list && !query.having_expression && !query.order_expression_list - && query.limit_length && !query_analyzer->hasAggregation() && limit_length + limit_offset < settings.max_block_size) + if (!query.distinct + && !query.prewhere_expression + && !query.where_expression + && !query.group_expression_list + && !query.having_expression + && !query.order_expression_list + && query.limit_length + && !query_analyzer->hasAggregation() + && limit_length + limit_offset < settings.max_block_size) { settings.max_block_size = limit_length + limit_offset; settings.max_threads = 1; @@ -678,12 +685,6 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu /// Инициализируем изначальные потоки данных, на которые накладываются преобразования запроса. Таблица или подзапрос? if (!interpreter_subquery) { - /** При распределённой обработке запроса, на все удалённые серверы отправляются временные таблицы, - * полученные из глобальных подзапросов - GLOBAL IN/JOIN. - */ - if (storage && storage->isRemote()) - storage->storeExternalTables(query_analyzer->getExternalTables()); - streams = storage->read(required_columns, query_ptr, context, settings_for_storage, from_stage, settings.max_block_size, settings.max_threads); @@ -817,12 +818,10 @@ static SortDescription getSortDescription(ASTSelectQuery & query) { SortDescription order_descr; order_descr.reserve(query.order_expression_list->children.size()); - for (ASTs::iterator it = query.order_expression_list->children.begin(); - it != query.order_expression_list->children.end(); - ++it) + for (const auto & elem : query.order_expression_list->children) { - String name = (*it)->children.front()->getColumnName(); - const ASTOrderByElement & order_by_elem = typeid_cast(**it); + String name = elem->children.front()->getColumnName(); + const ASTOrderByElement & order_by_elem = typeid_cast(*elem); order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.collator); } diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 6a090a38647..768a0581ea5 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -170,11 +170,20 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set) ConstColumnPlainPtrs key_columns(keys_size); data_types.resize(keys_size); + /// Константные столбцы справа от IN поддерживается не напрямую. Для этого, они сначала материализуется. + Columns materialized_columns; + /// Запоминаем столбцы, с которыми будем работать for (size_t i = 0; i < keys_size; ++i) { key_columns[i] = block.getByPosition(i).column; data_types[i] = block.getByPosition(i).type; + + if (key_columns[i]->isConst()) + { + materialized_columns.emplace_back(static_cast(key_columns[i])->convertToFullColumn()); + key_columns[i] = materialized_columns.back().get(); + } } size_t rows = block.rows(); diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index abb04fdb2b6..a1f60b3e5b3 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -83,7 +83,9 @@ void executeQuery( String query(begin, query_size); - LOG_DEBUG(&Logger::get("executeQuery"), query); + String logged_query = query; + std::replace(logged_query.begin(), logged_query.end(), '\n', ' '); + LOG_DEBUG(&Logger::get("executeQuery"), logged_query); /// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем. ProcessList::EntryPtr process_list_entry; diff --git a/dbms/src/Interpreters/tests/aggregate.cpp b/dbms/src/Interpreters/tests/aggregate.cpp index c838984ee62..ecac301b9fe 100644 --- a/dbms/src/Interpreters/tests/aggregate.cpp +++ b/dbms/src/Interpreters/tests/aggregate.cpp @@ -62,9 +62,9 @@ int main(int argc, char ** argv) DB::BlockInputStreamPtr stream = new DB::OneBlockInputStream(block); DB::AggregatedDataVariants aggregated_data_variants; - DB::ColumnNumbers key_column_numbers; - key_column_numbers.push_back(0); - key_column_numbers.push_back(1); + DB::Names key_column_names; + key_column_names.emplace_back("x"); + key_column_names.emplace_back("s1"); DB::AggregateFunctionFactory factory; @@ -73,7 +73,7 @@ int main(int argc, char ** argv) DB::DataTypes empty_list_of_types; aggregate_descriptions[0].function = factory.get("count", empty_list_of_types); - DB::Aggregator aggregator(key_column_numbers, aggregate_descriptions, false, 0, DB::OverflowMode::THROW, nullptr, 0, 0); + DB::Aggregator aggregator(key_column_names, aggregate_descriptions, false, 0, DB::OverflowMode::THROW, nullptr, 0, 0); { Poco::Stopwatch stopwatch; diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index 96434d0777e..04a15723b64 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -247,8 +247,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne catch (Exception & e) { std::stringstream s; - s << "Code: " << e.code() - << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what(); + s << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() + << ", Stack trace:\n\n" << e.getStackTrace().toString(); LOG_ERROR(log, s.str()); trySendExceptionToClient(s, request, response, used_output); } @@ -257,18 +257,21 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne std::stringstream s; s << "Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what(); + LOG_ERROR(log, s.str()); trySendExceptionToClient(s, request, response, used_output); } catch (std::exception & e) { std::stringstream s; s << "Code: " << ErrorCodes::STD_EXCEPTION << ". " << e.what(); + LOG_ERROR(log, s.str()); trySendExceptionToClient(s, request, response, used_output); } catch (...) { std::stringstream s; s << "Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ". Unknown exception."; + LOG_ERROR(log, s.str()); trySendExceptionToClient(s, request, response, used_output); } } diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 2b6e7a121bd..2f96d5e85c9 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -716,7 +716,7 @@ void TCPHandler::run() } catch (Exception & e) { - LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() + LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() << ", Stack trace:\n\n" << e.getStackTrace().toString()); } catch (Poco::Exception & e) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 468a0d2e0b8..68d09f1975f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -44,7 +44,7 @@ MergeTreeData::MergeTreeData( date_column_name(date_column_name_), sampling_expression(sampling_expression_), index_granularity(index_granularity_), mode(mode_), sign_column(sign_column_), columns_to_sum(columns_to_sum_), - settings(settings_), primary_expr_ast(primary_expr_ast_->clone()), + settings(settings_), primary_expr_ast(primary_expr_ast_ ? primary_expr_ast_->clone() : nullptr), require_part_metadata(require_part_metadata_), full_path(full_path_), columns(columns_), broken_part_callback(broken_part_callback_), @@ -89,18 +89,23 @@ MergeTreeData::MergeTreeData( Poco::File(full_path).createDirectories(); Poco::File(full_path + "detached").createDirectory(); - /// инициализируем описание сортировки - sort_descr.reserve(primary_expr_ast->children.size()); - for (const ASTPtr & ast : primary_expr_ast->children) + if (primary_expr_ast) { - String name = ast->getColumnName(); - sort_descr.push_back(SortColumnDescription(name, 1)); + /// инициализируем описание сортировки + sort_descr.reserve(primary_expr_ast->children.size()); + for (const ASTPtr & ast : primary_expr_ast->children) + { + String name = ast->getColumnName(); + sort_descr.push_back(SortColumnDescription(name, 1)); + } + + primary_expr = ExpressionAnalyzer(primary_expr_ast, context, getColumnsList()).getActions(false); + + ExpressionActionsPtr projected_expr = ExpressionAnalyzer(primary_expr_ast, context, getColumnsList()).getActions(true); + primary_key_sample = projected_expr->getSampleBlock(); } - - primary_expr = ExpressionAnalyzer(primary_expr_ast, context, getColumnsList()).getActions(false); - - ExpressionActionsPtr projected_expr = ExpressionAnalyzer(primary_expr_ast, context, getColumnsList()).getActions(true); - primary_key_sample = projected_expr->getSampleBlock(); + else if (mode != Unsorted) + throw Exception("Primary key could be empty only for UnsortedMergeTree", ErrorCodes::BAD_ARGUMENTS); } UInt64 MergeTreeData::getMaxDataPartIndex() @@ -120,6 +125,7 @@ std::string MergeTreeData::getModePrefix() const case Collapsing: return "Collapsing"; case Summing: return "Summing"; case Aggregating: return "Aggregating"; + case Unsorted: return "Unsorted"; default: throw Exception("Unknown mode of operation for MergeTreeData: " + toString(mode), ErrorCodes::LOGICAL_ERROR); @@ -386,8 +392,14 @@ void MergeTreeData::checkAlter(const AlterCommands & params) /// Список столбцов, которые нельзя трогать. /// sampling_expression можно не учитывать, потому что он обязан содержаться в первичном ключе. - Names keys = primary_expr->getRequiredColumns(); + + Names keys; + + if (primary_expr) + keys = primary_expr->getRequiredColumns(); + keys.push_back(sign_column); + std::sort(keys.begin(), keys.end()); for (const AlterCommand & command : params) @@ -510,7 +522,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( BlockInputStreamPtr part_in = new MergeTreeBlockInputStream(full_path + part->name + '/', DEFAULT_MERGE_BLOCK_SIZE, expression->getRequiredColumns(), *this, part, ranges, false, nullptr, "", false); ExpressionBlockInputStream in(part_in, expression); - MergedColumnOnlyOutputStream out(*this, full_path + part->name + '/', true); + MergedColumnOnlyOutputStream out(*this, full_path + part->name + '/', true, CompressionMethod::LZ4); in.readPrefix(); out.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index 5064169eb37..24f4286dac9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -335,8 +336,12 @@ MergeTreeData::DataPartPtr MergeTreeDataMerger::mergeParts( __sync_add_and_fetch(&merge_entry->bytes_read_uncompressed, value.bytes); }); - src_streams.push_back(new MaterializingBlockInputStream{ - new ExpressionBlockInputStream(input.release(), data.getPrimaryExpression())}); + if (data.mode != MergeTreeData::Unsorted) + src_streams.push_back(new MaterializingBlockInputStream{ + new ExpressionBlockInputStream(input.release(), data.getPrimaryExpression())}); + else + src_streams.push_back(input.release()); + sum_rows_approx += parts[i]->size * data.index_granularity; } @@ -363,13 +368,17 @@ MergeTreeData::DataPartPtr MergeTreeDataMerger::mergeParts( merged_stream = std::make_unique(src_streams, data.getSortDescription(), DEFAULT_MERGE_BLOCK_SIZE); break; + case MergeTreeData::Unsorted: + merged_stream = std::make_unique(src_streams); + break; + default: throw Exception("Unknown mode of operation for MergeTreeData: " + toString(data.mode), ErrorCodes::LOGICAL_ERROR); } const String new_part_tmp_path = data.getFullPath() + "tmp_" + merged_name + "/"; - MergedBlockOutputStream to{data, new_part_tmp_path, union_columns}; + MergedBlockOutputStream to{data, new_part_tmp_path, union_columns, CompressionMethod::LZ4}; merged_stream->readPrefix(); to.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 9374db261f2..1ddd78bb324 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -83,6 +83,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( PKCondition key_condition(query, context, data.getColumnsList(), data.getSortDescription()); PKCondition date_condition(query, context, data.getColumnsList(), SortDescription(1, SortColumnDescription(data.date_column_name, 1))); + if (settings.force_index_by_date && date_condition.alwaysTrue()) + throw Exception("Index by date is not used and setting 'force_index_by_date' is set.", ErrorCodes::INDEX_NOT_USED); + /// Выберем куски, в которых могут быть данные, удовлетворяющие date_condition, и которые подходят под условие на _part. { auto prev_parts = parts; @@ -268,7 +271,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( for (auto & part : parts) { RangesInDataPart ranges(part, (*part_index)++); - ranges.ranges = markRangesFromPkRange(part->index, key_condition, settings); + + if (data.mode != MergeTreeData::Unsorted) + ranges.ranges = markRangesFromPkRange(part->index, key_condition, settings); + else + ranges.ranges = MarkRanges{MarkRange{0, part->size}}; if (!ranges.ranges.empty()) { @@ -281,7 +288,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( } LOG_DEBUG(log, "Selected " << parts.size() << " parts by date, " << parts_with_ranges.size() << " parts by key, " - << sum_marks << " marks to read from " << sum_ranges << " ranges"); + << sum_marks << " marks to read from " << sum_ranges << " ranges"); BlockInputStreams res; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index aafa4f0af91..cf6d0d32af9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -94,15 +94,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa new_data_part->is_temp = true; /// Если для сортировки надо вычислить некоторые столбцы - делаем это. - data.getPrimaryExpression()->execute(block); + if (data.mode != MergeTreeData::Unsorted) + data.getPrimaryExpression()->execute(block); SortDescription sort_descr = data.getSortDescription(); /// Сортируем. - stableSortBlock(block, sort_descr); + if (data.mode != MergeTreeData::Unsorted) + stableSortBlock(block, sort_descr); NamesAndTypesList columns = data.getColumnsList().filter(block.getColumnsList().getNames()); - MergedBlockOutputStream out(data, part_tmp_path, columns); + MergedBlockOutputStream out(data, part_tmp_path, columns, CompressionMethod::LZ4); + out.getIndex().reserve(part_size * sort_descr.size()); out.writePrefix(); diff --git a/dbms/src/Storages/StorageChunkMerger.cpp b/dbms/src/Storages/StorageChunkMerger.cpp index bc0330819f6..f2879193771 100644 --- a/dbms/src/Storages/StorageChunkMerger.cpp +++ b/dbms/src/Storages/StorageChunkMerger.cpp @@ -21,10 +21,7 @@ namespace DB { - -const int SLEEP_AFTER_MERGE = 1; -const int SLEEP_NO_WORK = 10; -const int SLEEP_AFTER_ERROR = 60; +const int NOTHING_TO_MERGE_PERIOD = 10; StorageChunkMerger::TableNames StorageChunkMerger::currently_written_groups; @@ -226,6 +223,35 @@ Block StorageChunkMerger::getBlockWithVirtualColumns(const Storages & selected_t return res; } +class StorageChunkMerger::MergeTask +{ +public: + MergeTask(const StorageChunkMerger & chunk_merger_, DB::Context & context_, Logger * log_) + : + shutdown_called(false), + chunk_merger(chunk_merger_), + context(context_), + log(log_) + { + } + + bool merge(); + +public: + std::atomic shutdown_called; + +private: + bool maybeMergeSomething(); + Storages selectChunksToMerge(); + bool mergeChunks(const Storages & chunks); + + const StorageChunkMerger & chunk_merger; + DB::Context & context; + + Logger * log; + time_t last_nothing_to_merge_time = 0; +}; + StorageChunkMerger::StorageChunkMerger( const std::string & this_database_, const std::string & name_, @@ -242,20 +268,28 @@ StorageChunkMerger::StorageChunkMerger( this_database(this_database_), name(name_), columns(columns_), source_database(source_database_), table_name_regexp(table_name_regexp_), destination_name_prefix(destination_name_prefix_), chunks_to_merge(chunks_to_merge_), context(context_), settings(context.getSettings()), - log(&Logger::get("StorageChunkMerger")), shutdown_called(false) + log(&Logger::get("StorageChunkMerger")), + merge_task(std::make_shared(*this, context, log)) { - merge_thread = std::thread([this] { mergeThread(); }); _table_column_name = "_table" + VirtualColumnUtils::chooseSuffix(getColumnsList(), "_table"); + + auto & backgroud_pool = context.getBackgroundPool(); + MergeTaskPtr tmp_merge_task = merge_task; + DB::BackgroundProcessingPool::Task task = [tmp_merge_task](BackgroundProcessingPool::Context & pool_context) -> bool + { + return tmp_merge_task->merge(); + }; + + merge_task_handle = backgroud_pool.addTask(task); } void StorageChunkMerger::shutdown() { - if (shutdown_called) + if (merge_task->shutdown_called) return; - shutdown_called = true; - cancel_merge_thread.set(); - merge_thread.join(); + merge_task->shutdown_called.store(true); + context.getBackgroundPool().removeTask(merge_task_handle); } StorageChunkMerger::~StorageChunkMerger() @@ -263,41 +297,44 @@ StorageChunkMerger::~StorageChunkMerger() shutdown(); } -void StorageChunkMerger::mergeThread() +bool StorageChunkMerger::MergeTask::merge() { - while (true) + time_t now = time(0); + if (last_nothing_to_merge_time + NOTHING_TO_MERGE_PERIOD > now) + return false; + + if (shutdown_called) + return false; + + bool merged = false; + + try { - bool merged = false; - bool error = true; - - try - { - merged = maybeMergeSomething(); - error = false; - } - catch (const Exception & e) - { - LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() - << ", Stack trace:\n\n" << e.getStackTrace().toString()); - } - catch (const Poco::Exception & e) - { - LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() - << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()); - } - catch (const std::exception & e) - { - LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", e.what() = " << e.what()); - } - catch (...) - { - LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION); - } - - unsigned sleep_ammount = error ? SLEEP_AFTER_ERROR : (merged ? SLEEP_AFTER_MERGE : SLEEP_NO_WORK); - if (shutdown_called || cancel_merge_thread.tryWait(1000 * sleep_ammount)) - break; + merged = maybeMergeSomething(); } + catch (const Exception & e) + { + LOG_ERROR(log, "StorageChunkMerger at " << chunk_merger.this_database << "." << chunk_merger.name << " failed to merge: Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() + << ", Stack trace:\n\n" << e.getStackTrace().toString()); + } + catch (const Poco::Exception & e) + { + LOG_ERROR(log, "StorageChunkMerger at " << chunk_merger.this_database << "." << chunk_merger.name << " failed to merge: Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() + << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()); + } + catch (const std::exception & e) + { + LOG_ERROR(log, "StorageChunkMerger at " << chunk_merger.this_database << "." << chunk_merger.name << " failed to merge: std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", e.what() = " << e.what()); + } + catch (...) + { + LOG_ERROR(log, "StorageChunkMerger at " << chunk_merger.this_database << "." << chunk_merger.name << " failed to merge: unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION); + } + + if (!merged) + last_nothing_to_merge_time = now; + + return merged; } static std::string makeName(const std::string & prefix, const std::string & first_chunk, const std::string & last_chunk) @@ -308,7 +345,7 @@ static std::string makeName(const std::string & prefix, const std::string & firs return prefix + first_chunk + "_" + last_chunk.substr(lcp); } -bool StorageChunkMerger::maybeMergeSomething() +bool StorageChunkMerger::MergeTask::maybeMergeSomething() { Storages chunks = selectChunksToMerge(); if (chunks.empty() || shutdown_called) @@ -316,7 +353,7 @@ bool StorageChunkMerger::maybeMergeSomething() return mergeChunks(chunks); } -StorageChunkMerger::Storages StorageChunkMerger::selectChunksToMerge() +StorageChunkMerger::Storages StorageChunkMerger::MergeTask::selectChunksToMerge() { Poco::ScopedLock lock(context.getMutex()); @@ -324,26 +361,26 @@ StorageChunkMerger::Storages StorageChunkMerger::selectChunksToMerge() Databases & databases = context.getDatabases(); - if (!databases.count(source_database)) - throw Exception("No database " + source_database, ErrorCodes::UNKNOWN_DATABASE); + if (!databases.count(chunk_merger.source_database)) + throw Exception("No database " + chunk_merger.source_database, ErrorCodes::UNKNOWN_DATABASE); - Tables & tables = databases[source_database]; + Tables & tables = databases[chunk_merger.source_database]; for (Tables::iterator it = tables.begin(); it != tables.end(); ++it) { StoragePtr table = it->second; - if (table_name_regexp.match(it->first) && + if (chunk_merger.table_name_regexp.match(it->first) && !typeid_cast(&*table) && !typeid_cast(&*table) && !typeid_cast(&*table)) { res.push_back(table); - if (res.size() >= chunks_to_merge) + if (res.size() >= chunk_merger.chunks_to_merge) break; } } - if (res.size() < chunks_to_merge) + if (res.size() < chunk_merger.chunks_to_merge) res.clear(); return res; @@ -357,7 +394,7 @@ static ASTPtr newIdentifier(const std::string & name, ASTIdentifier::Kind kind) return res; } -bool StorageChunkMerger::mergeChunks(const Storages & chunks) +bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks) { typedef std::map ColumnsMap; @@ -371,11 +408,11 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) /// Объединим множества столбцов сливаемых чанков. ColumnsMap known_columns_types; - for (const NameAndTypePair & column : getColumnsList()) + for (const NameAndTypePair & column : chunk_merger.getColumnsList()) known_columns_types.insert(std::make_pair(column.name, column.type)); NamesAndTypesListPtr required_columns = new NamesAndTypesList; - *required_columns = *columns; + *required_columns = *chunk_merger.columns; for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) { @@ -402,8 +439,8 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) std::string formatted_columns = formatColumnsForCreateQuery(*required_columns); - std::string new_table_name = makeName(destination_name_prefix, chunks.front()->getTableName(), chunks.back()->getTableName()); - std::string new_table_full_name = backQuoteIfNeed(source_database) + "." + backQuoteIfNeed(new_table_name); + std::string new_table_name = makeName(chunk_merger.destination_name_prefix, chunks.front()->getTableName(), chunks.back()->getTableName()); + std::string new_table_full_name = backQuoteIfNeed(chunk_merger.source_database) + "." + backQuoteIfNeed(new_table_name); StoragePtr new_storage_ptr; try @@ -411,8 +448,8 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) { Poco::ScopedLock lock(context.getMutex()); - if (!context.getDatabases().count(source_database)) - throw Exception("Destination database " + source_database + " for table " + name + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); + if (!context.getDatabases().count(chunk_merger.source_database)) + throw Exception("Destination database " + chunk_merger.source_database + " for table " + chunk_merger.name + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); LOG_TRACE(log, "Will merge " << chunks.size() << " chunks: from " << chunks[0]->getTableName() << " to " << chunks.back()->getTableName() << " to new table " << new_table_name << "."); @@ -431,7 +468,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) /// Выполним запрос для создания Chunks таблицы. executeQuery("CREATE TABLE " + new_table_full_name + " " + formatted_columns + " ENGINE = Chunks", context, true); - new_storage_ptr = context.getTable(source_database, new_table_name); + new_storage_ptr = context.getTable(chunk_merger.source_database, new_table_name); /// Скопируем данные в новую таблицу. StorageChunks & new_storage = typeid_cast(*new_storage_ptr); @@ -451,7 +488,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) ASTPtr select_expression_list; ASTPtr database; ASTPtr table; /// Идентификатор или подзапрос (рекурсивно ASTSelectQuery) - select_query->database = newIdentifier(source_database, ASTIdentifier::Database); + select_query->database = newIdentifier(chunk_merger.source_database, ASTIdentifier::Database); select_query->table = newIdentifier(src_storage->getTableName(), ASTIdentifier::Table); ASTExpressionList * select_list = new ASTExpressionList; select_query->select_expression_list = select_list; @@ -467,7 +504,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) src_column_names, select_query_ptr, context, - settings, + chunk_merger.settings, processed_stage, DEFAULT_MERGE_BLOCK_SIZE); @@ -502,7 +539,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) Poco::ScopedLock lock(context.getMutex()); /// Если БД успели удалить, ничего не делаем. - if (context.getDatabases().count(source_database)) + if (context.getDatabases().count(chunk_merger.source_database)) { for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) { @@ -510,16 +547,16 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) std::string src_name = src_storage->getTableName(); /// Если таблицу успели удалить, ничего не делаем. - if (!context.isTableExist(source_database, src_name)) + if (!context.isTableExist(chunk_merger.source_database, src_name)) continue; /// Отцепляем исходную таблицу. Ее данные и метаданные остаются на диске. - tables_to_drop.push_back(context.detachTable(source_database, src_name)); + tables_to_drop.push_back(context.detachTable(chunk_merger.source_database, src_name)); /// Создаем на ее месте ChunkRef. Это возможно только потому что у ChunkRef нет ни, ни метаданных. try { - context.addTable(source_database, src_name, StorageChunkRef::create(src_name, context, source_database, new_table_name, false)); + context.addTable(chunk_merger.source_database, src_name, StorageChunkRef::create(src_name, context, chunk_merger.source_database, new_table_name, false)); } catch (...) { @@ -537,7 +574,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) table_locks.clear(); for (StoragePtr table : tables_to_drop) { - InterpreterDropQuery::dropDetachedTable(source_database, table, context); + InterpreterDropQuery::dropDetachedTable(chunk_merger.source_database, table, context); /// NOTE: Если между подменой таблицы и этой строчкой кто-то успеет попытаться создать новую таблицу на ее месте, /// что-нибудь может сломаться. } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index a52dd69b70b..39f0637f3bb 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -166,6 +166,8 @@ BlockInputStreams StorageDistributed::read( if (settings.limits.max_network_bandwidth) throttler.reset(new Throttler(settings.limits.max_network_bandwidth)); + Tables external_tables = context.getExternalTables(); + /// Цикл по шардам. for (auto & conn_pool : cluster.pools) res.emplace_back(new RemoteBlockInputStream{ @@ -177,9 +179,6 @@ BlockInputStreams StorageDistributed::read( { DB::Context new_context = context; new_context.setSettings(new_settings); - for (auto & it : external_tables) - if (!new_context.tryGetExternalTable(it.first)) - new_context.addExternalTable(it.first, it.second); for (size_t i = 0; i < cluster.getLocalNodesNum(); ++i) { @@ -193,7 +192,6 @@ BlockInputStreams StorageDistributed::read( } } - external_tables.clear(); return res; } diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 1a0ad57ed47..54aafbf66ad 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -350,7 +350,7 @@ StoragePtr StorageFactory::get( } else if (endsWith(name, "MergeTree")) { - /** Движки [Replicated][Summing|Collapsing|Aggregating|]MergeTree (8 комбинаций) + /** Движки [Replicated][|Summing|Collapsing|Aggregating|Unsorted]MergeTree (2 * 5 комбинаций) * В качестве аргумента для движка должно быть указано: * - (для Replicated) Путь к таблице в ZooKeeper * - (для Replicated) Имя реплики в ZooKeeper @@ -367,6 +367,7 @@ StoragePtr StorageFactory::get( * CollapsingMergeTree(date, [sample_key], primary_key, index_granularity, sign) * SummingMergeTree(date, [sample_key], primary_key, index_granularity, [columns_to_sum]) * AggregatingMergeTree(date, [sample_key], primary_key, index_granularity) + * UnsortedMergeTree(date, index_granularity) TODO Добавить описание ниже. */ const char * verbose_help = R"( @@ -445,6 +446,8 @@ For further info please read the documentation: http://clickhouse.yandex-team.ru mode = MergeTreeData::Summing; else if (name_part == "Aggregating") mode = MergeTreeData::Aggregating; + else if (name_part == "Unsorted") + mode = MergeTreeData::Unsorted; else if (!name_part.empty()) throw Exception("Unknown storage " + name + verbose_help, ErrorCodes::UNKNOWN_STORAGE); @@ -458,7 +461,26 @@ For further info please read the documentation: http://clickhouse.yandex-team.ru /// NOTE Слегка запутанно. size_t num_additional_params = (replicated ? 2 : 0) + (mode == MergeTreeData::Collapsing); - if (mode != MergeTreeData::Summing + if (mode == MergeTreeData::Unsorted + && args.size() != num_additional_params + 2) + { + String params; + + if (replicated) + params += + "\npath in ZooKeeper," + "\nreplica name,"; + + params += + "\nname of column with date," + "\nindex granularity\n"; + + throw Exception("Storage " + name + " requires " + + toString(num_additional_params + 2) + " parameters: " + params + verbose_help, + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + if (mode != MergeTreeData::Summing && mode != MergeTreeData::Unsorted && args.size() != num_additional_params + 3 && args.size() != num_additional_params + 4) { @@ -579,9 +601,10 @@ For further info please read the documentation: http://clickhouse.yandex-team.ru else throw Exception(String("Date column name must be an unquoted string") + verbose_help, ErrorCodes::BAD_ARGUMENTS); - primary_expr_list = extractPrimaryKey(args[1]); + if (mode != MergeTreeData::Unsorted) + primary_expr_list = extractPrimaryKey(args[1]); - auto ast = typeid_cast(&*args[2]); + auto ast = typeid_cast(&*args.back()); if (ast && ast->value.getType() == Field::Types::UInt64) index_granularity = safeGet(ast->value); else diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 6112eae172b..f99732892d9 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1058,28 +1058,6 @@ void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTr } LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << "."); - - if (unreplicated_data) - { - std::lock_guard unreplicated_lock(unreplicated_mutex); - - removed_parts = 0; - parts = unreplicated_data->getDataParts(); - for (const auto & part : parts) - { - if (!ActiveDataPartSet::contains(entry.new_part_name, part->name)) - continue; - LOG_DEBUG(log, "Removing unreplicated part " << part->name); - ++removed_parts; - - if (entry.detach) - unreplicated_data->renameAndDetachPart(part, ""); - else - unreplicated_data->replaceParts({part}, {}, false); - } - - LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << " (in unreplicated data)."); - } } diff --git a/dbms/tests/queries/0_stateless/00058_global_in.sh b/dbms/tests/queries/0_stateless/00058_global_in.sh index 446ee30e32e..a651e6893d6 100755 --- a/dbms/tests/queries/0_stateless/00058_global_in.sh +++ b/dbms/tests/queries/0_stateless/00058_global_in.sh @@ -12,6 +12,7 @@ fi # Создадим директории для данных второго сервера. PATH2=/tmp/clickhouse/ +[ -d "$PATH2" ] && rm -rf $PATH2 mkdir -p ${PATH2}{data,metadata}/default/ # Создадим второй конфиг с портом 9001. @@ -47,6 +48,7 @@ PID=$! function finish { kill $PID + wait } trap finish EXIT @@ -104,5 +106,3 @@ $CLIENT1 -n --query=" $CLIENT2 -n --query=" DROP TABLE test.half1; DROP TABLE test.half2;" - -rm -rf $PATH2 diff --git a/dbms/tests/queries/0_stateless/00134_aggregation_by_fixed_string_of_size_1_2_4_8.reference b/dbms/tests/queries/0_stateless/00134_aggregation_by_fixed_string_of_size_1_2_4_8.reference new file mode 100644 index 00000000000..5af20cfea5f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00134_aggregation_by_fixed_string_of_size_1_2_4_8.reference @@ -0,0 +1,9 @@ +\0 +\0\0 +\0\0\0 +\0\0\0\0 +\0\0\0\0\0 +\0\0\0\0\0\0 +\0\0\0\0\0\0\0 +\0\0\0\0\0\0\0\0 +\0\0\0\0\0\0\0\0\0 diff --git a/dbms/tests/queries/0_stateless/00134_aggregation_by_fixed_string_of_size_1_2_4_8.sql b/dbms/tests/queries/0_stateless/00134_aggregation_by_fixed_string_of_size_1_2_4_8.sql new file mode 100644 index 00000000000..4514b2597a7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00134_aggregation_by_fixed_string_of_size_1_2_4_8.sql @@ -0,0 +1,9 @@ +SELECT materialize(toFixedString('', 1)) AS x FROM system.one GROUP BY x; +SELECT materialize(toFixedString('', 2)) AS x FROM system.one GROUP BY x; +SELECT materialize(toFixedString('', 3)) AS x FROM system.one GROUP BY x; +SELECT materialize(toFixedString('', 4)) AS x FROM system.one GROUP BY x; +SELECT materialize(toFixedString('', 5)) AS x FROM system.one GROUP BY x; +SELECT materialize(toFixedString('', 6)) AS x FROM system.one GROUP BY x; +SELECT materialize(toFixedString('', 7)) AS x FROM system.one GROUP BY x; +SELECT materialize(toFixedString('', 8)) AS x FROM system.one GROUP BY x; +SELECT materialize(toFixedString('', 9)) AS x FROM system.one GROUP BY x; diff --git a/dbms/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.reference b/dbms/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.reference new file mode 100644 index 00000000000..a00ee0ba85e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.reference @@ -0,0 +1,256 @@ +0 0 0 +1 1 1 +10 10 10 +100 100 100 +101 101 101 +102 102 102 +103 103 103 +104 104 104 +105 105 105 +106 106 106 +107 107 107 +108 108 108 +109 109 109 +11 11 11 +110 110 110 +111 111 111 +112 112 112 +113 113 113 +114 114 114 +115 115 115 +116 116 116 +117 117 117 +118 118 118 +119 119 119 +12 12 12 +120 120 120 +121 121 121 +122 122 122 +123 123 123 +124 124 124 +125 125 125 +126 126 126 +127 127 127 +128 128 128 +129 129 129 +13 13 13 +130 130 130 +131 131 131 +132 132 132 +133 133 133 +134 134 134 +135 135 135 +136 136 136 +137 137 137 +138 138 138 +139 139 139 +14 14 14 +140 140 140 +141 141 141 +142 142 142 +143 143 143 +144 144 144 +145 145 145 +146 146 146 +147 147 147 +148 148 148 +149 149 149 +15 15 15 +150 150 150 +151 151 151 +152 152 152 +153 153 153 +154 154 154 +155 155 155 +156 156 156 +157 157 157 +158 158 158 +159 159 159 +16 16 16 +160 160 160 +161 161 161 +162 162 162 +163 163 163 +164 164 164 +165 165 165 +166 166 166 +167 167 167 +168 168 168 +169 169 169 +17 17 17 +170 170 170 +171 171 171 +172 172 172 +173 173 173 +174 174 174 +175 175 175 +176 176 176 +177 177 177 +178 178 178 +179 179 179 +18 18 18 +180 180 180 +181 181 181 +182 182 182 +183 183 183 +184 184 184 +185 185 185 +186 186 186 +187 187 187 +188 188 188 +189 189 189 +19 19 19 +190 190 190 +191 191 191 +192 192 192 +193 193 193 +194 194 194 +195 195 195 +196 196 196 +197 197 197 +198 198 198 +199 199 199 +2 2 2 +20 20 20 +200 200 200 +201 201 201 +202 202 202 +203 203 203 +204 204 204 +205 205 205 +206 206 206 +207 207 207 +208 208 208 +209 209 209 +21 21 21 +210 210 210 +211 211 211 +212 212 212 +213 213 213 +214 214 214 +215 215 215 +216 216 216 +217 217 217 +218 218 218 +219 219 219 +22 22 22 +220 220 220 +221 221 221 +222 222 222 +223 223 223 +224 224 224 +225 225 225 +226 226 226 +227 227 227 +228 228 228 +229 229 229 +23 23 23 +230 230 230 +231 231 231 +232 232 232 +233 233 233 +234 234 234 +235 235 235 +236 236 236 +237 237 237 +238 238 238 +239 239 239 +24 24 24 +240 240 240 +241 241 241 +242 242 242 +243 243 243 +244 244 244 +245 245 245 +246 246 246 +247 247 247 +248 248 248 +249 249 249 +25 25 25 +250 250 250 +251 251 251 +252 252 252 +253 253 253 +254 254 254 +255 255 255 +26 26 26 +27 27 27 +28 28 28 +29 29 29 +3 3 3 +30 30 30 +31 31 31 +32 32 32 +33 33 33 +34 34 34 +35 35 35 +36 36 36 +37 37 37 +38 38 38 +39 39 39 +4 4 4 +40 40 40 +41 41 41 +42 42 42 +43 43 43 +44 44 44 +45 45 45 +46 46 46 +47 47 47 +48 48 48 +49 49 49 +5 5 5 +50 50 50 +51 51 51 +52 52 52 +53 53 53 +54 54 54 +55 55 55 +56 56 56 +57 57 57 +58 58 58 +59 59 59 +6 6 6 +60 60 60 +61 61 61 +62 62 62 +63 63 63 +64 64 64 +65 65 65 +66 66 66 +67 67 67 +68 68 68 +69 69 69 +7 7 7 +70 70 70 +71 71 71 +72 72 72 +73 73 73 +74 74 74 +75 75 75 +76 76 76 +77 77 77 +78 78 78 +79 79 79 +8 8 8 +80 80 80 +81 81 81 +82 82 82 +83 83 83 +84 84 84 +85 85 85 +86 86 86 +87 87 87 +88 88 88 +89 89 89 +9 9 9 +90 90 90 +91 91 91 +92 92 92 +93 93 93 +94 94 94 +95 95 95 +96 96 96 +97 97 97 +98 98 98 +99 99 99 diff --git a/dbms/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.sql b/dbms/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.sql new file mode 100644 index 00000000000..16356046a36 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.sql @@ -0,0 +1,3 @@ +SET max_rows_to_read = 1000000; +SET read_overflow_mode = 'break'; +SELECT concat(toString(number % 256 AS n), '') AS s, n, max(s) FROM system.numbers_mt GROUP BY s, n, n, n, n, n, n, n, n, n ORDER BY s, n; diff --git a/dbms/tests/queries/0_stateless/00136_duplicate_order_by_elems.reference b/dbms/tests/queries/0_stateless/00136_duplicate_order_by_elems.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00136_duplicate_order_by_elems.sql b/dbms/tests/queries/0_stateless/00136_duplicate_order_by_elems.sql new file mode 100644 index 00000000000..66a0d1a11d5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00136_duplicate_order_by_elems.sql @@ -0,0 +1 @@ +SELECT n FROM (SELECT number AS n FROM system.numbers LIMIT 1000000) ORDER BY n, n, n, n, n, n, n, n, n, n LIMIT 1000000, 1; diff --git a/dbms/tests/queries/0_stateless/00137_in_constants.reference b/dbms/tests/queries/0_stateless/00137_in_constants.reference new file mode 100644 index 00000000000..379885fb1ab --- /dev/null +++ b/dbms/tests/queries/0_stateless/00137_in_constants.reference @@ -0,0 +1,30 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00137_in_constants.sql b/dbms/tests/queries/0_stateless/00137_in_constants.sql new file mode 100644 index 00000000000..297acc4ef26 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00137_in_constants.sql @@ -0,0 +1,31 @@ +SELECT 1 IN (SELECT 1); +SELECT materialize(1) IN (SELECT 1); +SELECT 1 IN (SELECT materialize(1)); +SELECT materialize(1) IN (SELECT materialize(1)); +SELECT (1, 2) IN (SELECT 1, 2); +SELECT (1, materialize(2)) IN (SELECT 1, 2); +SELECT (1, materialize(2)) IN (SELECT materialize(1), 2); +SELECT (1, materialize(2), 'Hello') IN (SELECT materialize(1), 2, 'Hello'); +SELECT (1, materialize(2), materialize('Hello')) IN (SELECT materialize(1), 2, 'Hello'); +SELECT (1, materialize(2), materialize('Hello')) IN (SELECT materialize(1), 2, materialize('Hello')); +SELECT (1, materialize(2), 'Hello') IN (SELECT materialize(1), 2, materialize('Hello')); +SELECT 'Hello' IN (SELECT 'Hello'); +SELECT materialize('Hello') IN (SELECT 'Hello'); +SELECT 'Hello' IN (SELECT materialize('Hello')); +SELECT materialize('Hello') IN (SELECT materialize('Hello')); + +SELECT 2 IN (SELECT 1); +SELECT materialize(2) IN (SELECT 1); +SELECT 2 IN (SELECT materialize(1)); +SELECT materialize(2) IN (SELECT materialize(1)); +SELECT (1, 3) IN (SELECT 1, 2); +SELECT (1, materialize(3)) IN (SELECT 1, 2); +SELECT (1, materialize(3)) IN (SELECT materialize(1), 2); +SELECT (1, materialize(2), 'World') IN (SELECT materialize(1), 2, 'Hello'); +SELECT (1, materialize(2), materialize('World')) IN (SELECT materialize(1), 2, 'Hello'); +SELECT (1, materialize(2), materialize('World')) IN (SELECT materialize(1), 2, materialize('Hello')); +SELECT (1, materialize(2), 'World') IN (SELECT materialize(1), 2, materialize('Hello')); +SELECT 'World' IN (SELECT 'Hello'); +SELECT materialize('World') IN (SELECT 'Hello'); +SELECT 'World' IN (SELECT materialize('Hello')); +SELECT materialize('World') IN (SELECT materialize('Hello')); diff --git a/libs/libmysqlxx/include/mysqlxx/Null.h b/libs/libmysqlxx/include/mysqlxx/Null.h index a518add9663..b142c0c0224 100644 --- a/libs/libmysqlxx/include/mysqlxx/Null.h +++ b/libs/libmysqlxx/include/mysqlxx/Null.h @@ -23,7 +23,7 @@ class Null public: T data; bool is_null; - + Null() : is_null(true) {} Null(NullType data) : is_null(true) {} explicit Null(const T & data_) : data(data_), is_null(false) {} @@ -68,7 +68,7 @@ public: bool operator== (const NullType other) const { return is_null; } - bool operator!= (const Null & other) const + bool operator!= (const Null & other) const { return !(*this == other); } @@ -82,4 +82,12 @@ public: }; +template +T getValueFromNull(const Null & maybe) +{ + if (maybe.isNull()) + return {}; + return maybe; +} + } diff --git a/libs/libzkutil/include/zkutil/Increment.h b/libs/libzkutil/include/zkutil/Increment.h new file mode 100644 index 00000000000..5cd8f02316c --- /dev/null +++ b/libs/libzkutil/include/zkutil/Increment.h @@ -0,0 +1,48 @@ +#pragma once + +#include + +namespace zkutil +{ + +class Increment +{ +public: + Increment(ZooKeeperPtr zk_, const std::string & path_) + : zk(zk_), path(path_) + { + zk->createAncestors(path); + } + + size_t get() + { + LOG_TRACE(log, "Get increment"); + + size_t result = 0; + std::string result_str; + zkutil::Stat stat; + + bool success = false; + do + { + if (zk->tryGet(path, result_str, &stat)) + { + result = std::stol(result_str) + 1; + success = zk->trySet(path, std::to_string(result), stat.version) == ZOK; + } + else + { + success = zk->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZOK; + } + } + while(!success); + + return result; + } +private: + zkutil::ZooKeeperPtr zk; + std::string path; + Logger * log = &Logger::get("zkutil::Increment"); +}; + +} \ No newline at end of file diff --git a/libs/libzstd/README b/libs/libzstd/README new file mode 100644 index 00000000000..2d980b1341a --- /dev/null +++ b/libs/libzstd/README @@ -0,0 +1 @@ +https://github.com/Cyan4973/zstd/tree/765207c54934d478488c236749b01c7d6fc63d70/ diff --git a/libs/libzstd/include/zstd/LICENSE b/libs/libzstd/include/zstd/LICENSE new file mode 100644 index 00000000000..35495850f2e --- /dev/null +++ b/libs/libzstd/include/zstd/LICENSE @@ -0,0 +1,26 @@ +ZSTD Library +Copyright (c) 2014-2015, Yann Collet +All rights reserved. + +BSD License + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/libs/libzstd/include/zstd/fse.c b/libs/libzstd/include/zstd/fse.c new file mode 100644 index 00000000000..e07b951cdec --- /dev/null +++ b/libs/libzstd/include/zstd/fse.c @@ -0,0 +1,1613 @@ +/* ****************************************************************** + FSE : Finite State Entropy coder + Copyright (C) 2013-2015, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - FSE source repository : https://github.com/Cyan4973/FiniteStateEntropy + - Public forum : https://groups.google.com/forum/#!forum/lz4c +****************************************************************** */ + +#ifndef FSE_COMMONDEFS_ONLY + +/**************************************************************** +* Tuning parameters +****************************************************************/ +/* MEMORY_USAGE : +* Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.) +* Increasing memory usage improves compression ratio +* Reduced memory usage can improve speed, due to cache effect +* Recommended max value is 14, for 16KB, which nicely fits into Intel x86 L1 cache */ +#define FSE_MAX_MEMORY_USAGE 14 +#define FSE_DEFAULT_MEMORY_USAGE 13 + +/* FSE_MAX_SYMBOL_VALUE : +* Maximum symbol value authorized. +* Required for proper stack allocation */ +#define FSE_MAX_SYMBOL_VALUE 255 + + +/**************************************************************** +* Generic function type & suffix (C template emulation) +****************************************************************/ +#define FSE_FUNCTION_TYPE BYTE +#define FSE_FUNCTION_EXTENSION + +#endif /* !FSE_COMMONDEFS_ONLY */ + + +/**************************************************************** +* Compiler specifics +****************************************************************/ +#ifdef _MSC_VER /* Visual Studio */ +# define FORCE_INLINE static __forceinline +# include /* For Visual 2005 */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# pragma warning(disable : 4214) /* disable: C4214: non-int bitfields */ +#else +# define GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) +# ifdef __GNUC__ +# define FORCE_INLINE static inline __attribute__((always_inline)) +# else +# define FORCE_INLINE static inline +# endif +#endif + + +/**************************************************************** +* Includes +****************************************************************/ +#include /* malloc, free, qsort */ +#include /* memcpy, memset */ +#include /* printf (debug) */ +#include "fse_static.h" + + +#ifndef MEM_ACCESS_MODULE +#define MEM_ACCESS_MODULE +/**************************************************************** +* Basic Types +*****************************************************************/ +#if defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ +# include +typedef uint8_t BYTE; +typedef uint16_t U16; +typedef int16_t S16; +typedef uint32_t U32; +typedef int32_t S32; +typedef uint64_t U64; +typedef int64_t S64; +#else +typedef unsigned char BYTE; +typedef unsigned short U16; +typedef signed short S16; +typedef unsigned int U32; +typedef signed int S32; +typedef unsigned long long U64; +typedef signed long long S64; +#endif + +#endif /* MEM_ACCESS_MODULE */ + +/**************************************************************** +* Memory I/O +*****************************************************************/ +static unsigned FSE_isLittleEndian(void) +{ + const union { U32 i; BYTE c[4]; } one = { 1 }; /* don't use static : performance detrimental */ + return one.c[0]; +} + +static U32 FSE_read32(const void* memPtr) +{ + U32 val32; + memcpy(&val32, memPtr, 4); + return val32; +} + +static U32 FSE_readLE32(const void* memPtr) +{ + if (FSE_isLittleEndian()) + return FSE_read32(memPtr); + else + { + const BYTE* p = (const BYTE*)memPtr; + return (U32)((U32)p[0] + ((U32)p[1]<<8) + ((U32)p[2]<<16) + ((U32)p[3]<<24)); + } +} + +static void FSE_writeLE32(void* memPtr, U32 val32) +{ + if (FSE_isLittleEndian()) + { + memcpy(memPtr, &val32, 4); + } + else + { + BYTE* p = (BYTE*)memPtr; + p[0] = (BYTE)val32; + p[1] = (BYTE)(val32>>8); + p[2] = (BYTE)(val32>>16); + p[3] = (BYTE)(val32>>24); + } +} + +static U64 FSE_read64(const void* memPtr) +{ + U64 val64; + memcpy(&val64, memPtr, 8); + return val64; +} + +static U64 FSE_readLE64(const void* memPtr) +{ + if (FSE_isLittleEndian()) + return FSE_read64(memPtr); + else + { + const BYTE* p = (const BYTE*)memPtr; + return (U64)((U64)p[0] + ((U64)p[1]<<8) + ((U64)p[2]<<16) + ((U64)p[3]<<24) + + ((U64)p[4]<<32) + ((U64)p[5]<<40) + ((U64)p[6]<<48) + ((U64)p[7]<<56)); + } +} + +static void FSE_writeLE64(void* memPtr, U64 val64) +{ + if (FSE_isLittleEndian()) + { + memcpy(memPtr, &val64, 8); + } + else + { + BYTE* p = (BYTE*)memPtr; + p[0] = (BYTE)val64; + p[1] = (BYTE)(val64>>8); + p[2] = (BYTE)(val64>>16); + p[3] = (BYTE)(val64>>24); + p[4] = (BYTE)(val64>>32); + p[5] = (BYTE)(val64>>40); + p[6] = (BYTE)(val64>>48); + p[7] = (BYTE)(val64>>56); + } +} + +static size_t FSE_readLEST(const void* memPtr) +{ + if (sizeof(size_t)==4) + return (size_t)FSE_readLE32(memPtr); + else + return (size_t)FSE_readLE64(memPtr); +} + +static void FSE_writeLEST(void* memPtr, size_t val) +{ + if (sizeof(size_t)==4) + FSE_writeLE32(memPtr, (U32)val); + else + FSE_writeLE64(memPtr, (U64)val); +} + + +/**************************************************************** +* Constants +*****************************************************************/ +#define FSE_MAX_TABLELOG (FSE_MAX_MEMORY_USAGE-2) +#define FSE_MAX_TABLESIZE (1U< FSE_TABLELOG_ABSOLUTE_MAX +#error "FSE_MAX_TABLELOG > FSE_TABLELOG_ABSOLUTE_MAX is not supported" +#endif + + +/**************************************************************** +* Error Management +****************************************************************/ +#define FSE_STATIC_ASSERT(c) { enum { FSE_static_assert = 1/(int)(!!(c)) }; } /* use only *after* variable declarations */ + + +/**************************************************************** +* Complex types +****************************************************************/ +typedef struct +{ + int deltaFindState; + U16 maxState; + BYTE minBitsOut; + /* one byte padding */ +} FSE_symbolCompressionTransform; + +typedef struct +{ + U32 fakeTable[FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)]; /* compatible with FSE_compressU16() */ +} CTable_max_t; + + +/**************************************************************** +* Internal functions +****************************************************************/ +FORCE_INLINE unsigned FSE_highbit32 (register U32 val) +{ +# if defined(_MSC_VER) /* Visual */ + unsigned long r; + _BitScanReverse ( &r, val ); + return (unsigned) r; +# elif defined(__GNUC__) && (GCC_VERSION >= 304) /* GCC Intrinsic */ + return 31 - __builtin_clz (val); +# else /* Software version */ + static const unsigned DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 }; + U32 v = val; + unsigned r; + v |= v >> 1; + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + r = DeBruijnClz[ (U32) (v * 0x07C4ACDDU) >> 27]; + return r; +# endif +} + + +#ifndef FSE_COMMONDEFS_ONLY + +unsigned FSE_isError(size_t code) { return (code > (size_t)(-FSE_ERROR_maxCode)); } + +#define FSE_GENERATE_STRING(STRING) #STRING, +static const char* FSE_errorStrings[] = { FSE_LIST_ERRORS(FSE_GENERATE_STRING) }; + +const char* FSE_getErrorName(size_t code) +{ + static const char* codeError = "Unspecified error code"; + if (FSE_isError(code)) return FSE_errorStrings[-(int)(code)]; + return codeError; +} + +static short FSE_abs(short a) +{ + return a<0? -a : a; +} + + +/**************************************************************** +* Header bitstream management +****************************************************************/ +size_t FSE_headerBound(unsigned maxSymbolValue, unsigned tableLog) +{ + size_t maxHeaderSize = (((maxSymbolValue+1) * tableLog) >> 3) + 1; + return maxSymbolValue ? maxHeaderSize : FSE_MAX_HEADERSIZE; +} + +static size_t FSE_writeHeader_generic (void* header, size_t headerBufferSize, + const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog, + unsigned safeWrite) +{ + BYTE* const ostart = (BYTE*) header; + BYTE* out = ostart; + BYTE* const oend = ostart + headerBufferSize; + int nbBits; + const int tableSize = 1 << tableLog; + int remaining; + int threshold; + U32 bitStream; + int bitCount; + unsigned charnum = 0; + int previous0 = 0; + + bitStream = 0; + bitCount = 0; + /* Table Size */ + bitStream += (tableLog-FSE_MIN_TABLELOG) << bitCount; + bitCount += 4; + + /* Init */ + remaining = tableSize+1; /* +1 for extra accuracy */ + threshold = tableSize; + nbBits = tableLog+1; + + while (remaining>1) /* stops at 1 */ + { + if (previous0) + { + unsigned start = charnum; + while (!normalizedCounter[charnum]) charnum++; + while (charnum >= start+24) + { + start+=24; + bitStream += 0xFFFF< oend-2)) return (size_t)-FSE_ERROR_GENERIC; /* Buffer overflow */ + out[0] = (BYTE)bitStream; + out[1] = (BYTE)(bitStream>>8); + out+=2; + bitStream>>=16; + } + while (charnum >= start+3) + { + start+=3; + bitStream += 3 << bitCount; + bitCount += 2; + } + bitStream += (charnum-start) << bitCount; + bitCount += 2; + if (bitCount>16) + { + if ((!safeWrite) && (out > oend - 2)) return (size_t)-FSE_ERROR_GENERIC; /* Buffer overflow */ + out[0] = (BYTE)bitStream; + out[1] = (BYTE)(bitStream>>8); + out += 2; + bitStream >>= 16; + bitCount -= 16; + } + } + { + short count = normalizedCounter[charnum++]; + const short max = (short)((2*threshold-1)-remaining); + remaining -= FSE_abs(count); + if (remaining<0) return (size_t)-FSE_ERROR_GENERIC; + count++; /* +1 for extra accuracy */ + if (count>=threshold) count += max; /* [0..max[ [max..threshold[ (...) [threshold+max 2*threshold[ */ + bitStream += count << bitCount; + bitCount += nbBits; + bitCount -= (count>=1; + } + if (bitCount>16) + { + if ((!safeWrite) && (out > oend - 2)) return (size_t)-FSE_ERROR_GENERIC; /* Buffer overflow */ + out[0] = (BYTE)bitStream; + out[1] = (BYTE)(bitStream>>8); + out += 2; + bitStream >>= 16; + bitCount -= 16; + } + } + + /* flush remaining bitStream */ + if ((!safeWrite) && (out > oend - 2)) return (size_t)-FSE_ERROR_GENERIC; /* Buffer overflow */ + out[0] = (BYTE)bitStream; + out[1] = (BYTE)(bitStream>>8); + out+= (bitCount+7) /8; + + if (charnum > maxSymbolValue + 1) return (size_t)-FSE_ERROR_GENERIC; /* Too many symbols written (a bit too late?) */ + + return (out-ostart); +} + + +size_t FSE_writeHeader (void* header, size_t headerBufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog) +{ + if (tableLog > FSE_MAX_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported */ + if (tableLog < FSE_MIN_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported */ + + if (headerBufferSize < FSE_headerBound(maxSymbolValue, tableLog)) + return FSE_writeHeader_generic(header, headerBufferSize, normalizedCounter, maxSymbolValue, tableLog, 0); + + return FSE_writeHeader_generic(header, headerBufferSize, normalizedCounter, maxSymbolValue, tableLog, 1); +} + + +size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSVPtr, unsigned* tableLogPtr, + const void* headerBuffer, size_t hbSize) +{ + const BYTE* const istart = (const BYTE*) headerBuffer; + const BYTE* ip = istart; + int nbBits; + int remaining; + int threshold; + U32 bitStream; + int bitCount; + unsigned charnum = 0; + int previous0 = 0; + + bitStream = FSE_readLE32(ip); + nbBits = (bitStream & 0xF) + FSE_MIN_TABLELOG; /* extract tableLog */ + if (nbBits > FSE_TABLELOG_ABSOLUTE_MAX) return (size_t)-FSE_ERROR_tableLog_tooLarge; + bitStream >>= 4; + bitCount = 4; + *tableLogPtr = nbBits; + remaining = (1<1) && (charnum<=*maxSVPtr)) + { + if (previous0) + { + unsigned n0 = charnum; + while ((bitStream & 0xFFFF) == 0xFFFF) + { + n0+=24; + ip+=2; + bitStream = FSE_readLE32(ip) >> bitCount; + } + while ((bitStream & 3) == 3) + { + n0+=3; + bitStream>>=2; + bitCount+=2; + } + n0 += bitStream & 3; + bitCount += 2; + if (n0 > *maxSVPtr) return (size_t)-FSE_ERROR_GENERIC; + while (charnum < n0) normalizedCounter[charnum++] = 0; + ip += bitCount>>3; + bitCount &= 7; + bitStream = FSE_readLE32(ip) >> bitCount; + } + { + const short max = (short)((2*threshold-1)-remaining); + short count; + + if ((bitStream & (threshold-1)) < (U32)max) + { + count = (short)(bitStream & (threshold-1)); + bitCount += nbBits-1; + } + else + { + count = (short)(bitStream & (2*threshold-1)); + if (count >= threshold) count -= max; + bitCount += nbBits; + } + + count--; /* extra accuracy */ + remaining -= FSE_abs(count); + normalizedCounter[charnum++] = count; + previous0 = !count; + while (remaining < threshold) + { + nbBits--; + threshold >>= 1; + } + + ip += bitCount>>3; + bitCount &= 7; + bitStream = FSE_readLE32(ip) >> bitCount; + } + } + if (remaining != 1) return (size_t)-FSE_ERROR_GENERIC; + *maxSVPtr = charnum-1; + + ip += bitCount>0; + if ((size_t)(ip-istart) >= hbSize) return (size_t)-FSE_ERROR_srcSize_wrong; /* arguably a bit late , tbd */ + return ip-istart; +} + + +/**************************************************************** +* FSE Compression Code +****************************************************************/ +/* +CTable is a variable size structure which contains : + U16 tableLog; + U16 maxSymbolValue; + U16 nextStateNumber[1 << tableLog]; // This size is variable + FSE_symbolCompressionTransform symbolTT[maxSymbolValue+1]; // This size is variable +Allocation is manual, since C standard does not support variable-size structures. +*/ + +size_t FSE_sizeof_CTable (unsigned maxSymbolValue, unsigned tableLog) +{ + size_t size; + FSE_STATIC_ASSERT((size_t)FSE_CTABLE_SIZE_U32(FSE_MAX_TABLELOG, FSE_MAX_SYMBOL_VALUE)*4 >= sizeof(CTable_max_t)); /* A compilation error here means FSE_CTABLE_SIZE_U32 is not large enough */ + if (tableLog > FSE_MAX_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; + size = FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32); + return size; +} + +void* FSE_createCTable (unsigned maxSymbolValue, unsigned tableLog) +{ + size_t size; + if (tableLog > FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX; + size = FSE_CTABLE_SIZE_U32 (tableLog, maxSymbolValue) * sizeof(U32); + return malloc(size); +} + +void FSE_freeCTable (void* CTable) +{ + free(CTable); +} + + +unsigned FSE_optimalTableLog(unsigned maxTableLog, size_t srcSize, unsigned maxSymbolValue) +{ + U32 tableLog = maxTableLog; + if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG; + if ((FSE_highbit32((U32)(srcSize - 1)) - 2) < tableLog) tableLog = FSE_highbit32((U32)(srcSize - 1)) - 2; /* Accuracy can be reduced */ + if ((FSE_highbit32(maxSymbolValue+1)+1) > tableLog) tableLog = FSE_highbit32(maxSymbolValue+1)+1; /* Need a minimum to safely represent all symbol values */ + if (tableLog < FSE_MIN_TABLELOG) tableLog = FSE_MIN_TABLELOG; + if (tableLog > FSE_MAX_TABLELOG) tableLog = FSE_MAX_TABLELOG; + return tableLog; +} + + +typedef struct +{ + U32 id; + U32 count; +} rank_t; + +int FSE_compareRankT(const void* r1, const void* r2) +{ + const rank_t* R1 = (const rank_t*)r1; + const rank_t* R2 = (const rank_t*)r2; + + return 2 * (R1->count < R2->count) - 1; +} + + +#if 0 +static size_t FSE_adjustNormSlow(short* norm, int pointsToRemove, const unsigned* count, U32 maxSymbolValue) +{ + rank_t rank[FSE_MAX_SYMBOL_VALUE+2]; + U32 s; + + /* Init */ + for (s=0; s<=maxSymbolValue; s++) + { + rank[s].id = s; + rank[s].count = count[s]; + if (norm[s] <= 1) rank[s].count = 0; + } + rank[maxSymbolValue+1].id = 0; + rank[maxSymbolValue+1].count = 0; /* ensures comparison ends here in worst case */ + + /* Sort according to count */ + qsort(rank, maxSymbolValue+1, sizeof(rank_t), FSE_compareRankT); + + while(pointsToRemove) + { + int newRank = 1; + rank_t savedR; + if (norm[rank[0].id] == 1) + return (size_t)-FSE_ERROR_GENERIC; + norm[rank[0].id]--; + pointsToRemove--; + rank[0].count -= (rank[0].count + 6) >> 3; + if (norm[rank[0].id] == 1) + rank[0].count=0; + savedR = rank[0]; + while (rank[newRank].count > savedR.count) + { + rank[newRank-1] = rank[newRank]; + newRank++; + } + rank[newRank-1] = savedR; + } + + return 0; +} + +#else + +/* Secondary normalization method. + To be used when primary method fails. */ + +static size_t FSE_normalizeM2(short* norm, U32 tableLog, const unsigned* count, size_t total, U32 maxSymbolValue) +{ + U32 s; + U32 distributed = 0; + U32 ToDistribute; + + /* Init */ + U32 lowThreshold = (U32)(total >> tableLog); + U32 lowOne = (U32)((total * 3) >> (tableLog + 1)); + + for (s=0; s<=maxSymbolValue; s++) + { + if (count[s] == 0) + { + norm[s]=0; + continue; + } + if (count[s] <= lowThreshold) + { + norm[s] = -1; + distributed++; + total -= count[s]; + continue; + } + if (count[s] <= lowOne) + { + norm[s] = 1; + distributed++; + total -= count[s]; + continue; + } + norm[s]=-2; + } + ToDistribute = (1 << tableLog) - distributed; + + if ((total / ToDistribute) > lowOne) + { + /* risk of rounding to zero */ + lowOne = (U32)((total * 3) / (ToDistribute * 2)); + for (s=0; s<=maxSymbolValue; s++) + { + if ((norm[s] == -2) && (count[s] <= lowOne)) + { + norm[s] = 1; + distributed++; + total -= count[s]; + continue; + } + } + ToDistribute = (1 << tableLog) - distributed; + } + + if (distributed == maxSymbolValue+1) + { + /* all values are pretty poor; + probably incompressible data (should have already been detected); + find max, then give all remaining points to max */ + U32 maxV = 0, maxC =0; + for (s=0; s<=maxSymbolValue; s++) + if (count[s] > maxC) maxV=s, maxC=count[s]; + norm[maxV] += ToDistribute; + return 0; + } + + { + U64 const vStepLog = 62 - tableLog; + U64 const mid = (1ULL << (vStepLog-1)) - 1; + U64 const rStep = ((((U64)1<> vStepLog); + U32 sEnd = (U32)(end >> vStepLog); + U32 weight = sEnd - sStart; + if (weight < 1) + return (size_t)-FSE_ERROR_GENERIC; + norm[s] = weight; + tmpTotal = end; + } + } + } + + return 0; +} +#endif + + +size_t FSE_normalizeCount (short* normalizedCounter, unsigned tableLog, + const unsigned* count, size_t total, + unsigned maxSymbolValue) +{ + /* Sanity checks */ + if (tableLog==0) tableLog = FSE_DEFAULT_TABLELOG; + if (tableLog < FSE_MIN_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported size */ + if (tableLog > FSE_MAX_TABLELOG) return (size_t)-FSE_ERROR_GENERIC; /* Unsupported size */ + if ((1U<> tableLog); + + for (s=0; s<=maxSymbolValue; s++) + { + if (count[s] == total) return 0; + if (count[s] == 0) + { + normalizedCounter[s]=0; + continue; + } + if (count[s] <= lowThreshold) + { + normalizedCounter[s] = -1; + stillToDistribute--; + } + else + { + short proba = (short)((count[s]*step) >> scale); + if (proba<8) + { + U64 restToBeat = vStep * rtbTable[proba]; + proba += (count[s]*step) - ((U64)proba< restToBeat; + } + if (proba > largestP) + { + largestP=proba; + largest=s; + } + normalizedCounter[s] = proba; + stillToDistribute -= proba; + } + } + if (-stillToDistribute >= (normalizedCounter[largest] >> 1)) + { + /* corner case, need another normalization method */ + size_t errorCode = FSE_normalizeM2(normalizedCounter, tableLog, count, total, maxSymbolValue); + if (FSE_isError(errorCode)) return errorCode; + } + else normalizedCounter[largest] += (short)stillToDistribute; + } + +#if 0 + { /* Print Table (debug) */ + U32 s; + U32 nTotal = 0; + for (s=0; s<=maxSymbolValue; s++) + printf("%3i: %4i \n", s, normalizedCounter[s]); + for (s=0; s<=maxSymbolValue; s++) + nTotal += abs(normalizedCounter[s]); + if (nTotal != (1U<>1)); + unsigned s; + + /* Sanity checks */ + if (nbBits < 1) return (size_t)-FSE_ERROR_GENERIC; /* min size */ + if (((size_t)CTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */ + + /* header */ + tableU16[-2] = (U16) nbBits; + tableU16[-1] = (U16) maxSymbolValue; + + /* Build table */ + for (s=0; sbitContainer = 0; + bitC->bitPos = 0; /* reserved for unusedBits */ + bitC->startPtr = (char*)start; + bitC->ptr = bitC->startPtr; +} + +void FSE_initCState(FSE_CState_t* statePtr, const void* CTable) +{ + const U32 tableLog = ( (U16*) CTable) [0]; + statePtr->value = (ptrdiff_t)1<stateTable = ((const U16*) CTable) + 2; + statePtr->symbolTT = (const U32*)CTable + 1 + (tableLog ? (1<<(tableLog-1)) : 1); + statePtr->stateLog = tableLog; +} + +void FSE_addBits(FSE_CStream_t* bitC, size_t value, unsigned nbBits) +{ + static const unsigned mask[] = { 0, 1, 3, 7, 0xF, 0x1F, 0x3F, 0x7F, 0xFF, 0x1FF, 0x3FF, 0x7FF, 0xFFF, 0x1FFF, 0x3FFF, 0x7FFF, 0xFFFF, 0x1FFFF, 0x3FFFF, 0x7FFFF, 0xFFFFF, 0x1FFFFF, 0x3FFFFF, 0x7FFFFF, 0xFFFFFF, 0x1FFFFFF }; /* up to 25 bits */ + bitC->bitContainer |= (value & mask[nbBits]) << bitC->bitPos; + bitC->bitPos += nbBits; +} + +void FSE_encodeByte(FSE_CStream_t* bitC, FSE_CState_t* statePtr, BYTE symbol) +{ + const FSE_symbolCompressionTransform* const symbolTT = (const FSE_symbolCompressionTransform*) statePtr->symbolTT; + const U16* const stateTable = (const U16*) statePtr->stateTable; + int nbBitsOut = symbolTT[symbol].minBitsOut; + nbBitsOut -= (int)((symbolTT[symbol].maxState - statePtr->value) >> 31); + FSE_addBits(bitC, statePtr->value, nbBitsOut); + statePtr->value = stateTable[ (statePtr->value >> nbBitsOut) + symbolTT[symbol].deltaFindState]; +} + +void FSE_flushBits(FSE_CStream_t* bitC) +{ + size_t nbBytes = bitC->bitPos >> 3; + FSE_writeLEST(bitC->ptr, bitC->bitContainer); + bitC->bitPos &= 7; + bitC->ptr += nbBytes; + bitC->bitContainer >>= nbBytes*8; +} + +void FSE_flushCState(FSE_CStream_t* bitC, const FSE_CState_t* statePtr) +{ + FSE_addBits(bitC, statePtr->value, statePtr->stateLog); + FSE_flushBits(bitC); +} + + +size_t FSE_closeCStream(FSE_CStream_t* bitC) +{ + char* endPtr; + + FSE_addBits(bitC, 1, 1); + FSE_flushBits(bitC); + + endPtr = bitC->ptr; + endPtr += bitC->bitPos > 0; + + return (endPtr - bitC->startPtr); +} + + +size_t FSE_compress_usingCTable (void* dst, size_t dstSize, + const void* src, size_t srcSize, + const void* CTable) +{ + const BYTE* const istart = (const BYTE*) src; + const BYTE* ip; + const BYTE* const iend = istart + srcSize; + + FSE_CStream_t bitC; + FSE_CState_t CState1, CState2; + + + /* init */ + (void)dstSize; /* objective : ensure it fits into dstBuffer (Todo) */ + FSE_initCStream(&bitC, dst); + FSE_initCState(&CState1, CTable); + CState2 = CState1; + + ip=iend; + + /* join to even */ + if (srcSize & 1) + { + FSE_encodeByte(&bitC, &CState1, *--ip); + FSE_flushBits(&bitC); + } + + /* join to mod 4 */ + if ((sizeof(size_t)*8 > FSE_MAX_TABLELOG*4+7 ) && (srcSize & 2)) /* test bit 2 */ + { + FSE_encodeByte(&bitC, &CState2, *--ip); + FSE_encodeByte(&bitC, &CState1, *--ip); + FSE_flushBits(&bitC); + } + + /* 2 or 4 encoding per loop */ + while (ip>istart) + { + FSE_encodeByte(&bitC, &CState2, *--ip); + + if (sizeof(size_t)*8 < FSE_MAX_TABLELOG*2+7 ) /* this test must be static */ + FSE_flushBits(&bitC); + + FSE_encodeByte(&bitC, &CState1, *--ip); + + if (sizeof(size_t)*8 > FSE_MAX_TABLELOG*4+7 ) /* this test must be static */ + { + FSE_encodeByte(&bitC, &CState2, *--ip); + FSE_encodeByte(&bitC, &CState1, *--ip); + } + + FSE_flushBits(&bitC); + } + + FSE_flushCState(&bitC, &CState2); + FSE_flushCState(&bitC, &CState1); + return FSE_closeCStream(&bitC); +} + + +size_t FSE_compressBound(size_t size) { return FSE_COMPRESSBOUND(size); } + + +size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog) +{ + const BYTE* const istart = (const BYTE*) src; + const BYTE* ip = istart; + + BYTE* const ostart = (BYTE*) dst; + BYTE* op = ostart; + BYTE* const oend = ostart + dstSize; + + U32 count[FSE_MAX_SYMBOL_VALUE+1]; + S16 norm[FSE_MAX_SYMBOL_VALUE+1]; + CTable_max_t CTable; + size_t errorCode; + + /* early out */ + if (dstSize < FSE_compressBound(srcSize)) return (size_t)-FSE_ERROR_dstSize_tooSmall; + if (srcSize <= 1) return srcSize; /* Uncompressed or RLE */ + if (!maxSymbolValue) maxSymbolValue = FSE_MAX_SYMBOL_VALUE; + if (!tableLog) tableLog = FSE_DEFAULT_TABLELOG; + + /* Scan input and build symbol stats */ + errorCode = FSE_count (count, ip, srcSize, &maxSymbolValue); + if (FSE_isError(errorCode)) return errorCode; + if (errorCode == srcSize) return 1; + if (errorCode < (srcSize >> 7)) return 0; /* Heuristic : not compressible enough */ + + tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue); + errorCode = FSE_normalizeCount (norm, tableLog, count, srcSize, maxSymbolValue); + if (FSE_isError(errorCode)) return errorCode; + + /* Write table description header */ + errorCode = FSE_writeHeader (op, FSE_MAX_HEADERSIZE, norm, maxSymbolValue, tableLog); + if (FSE_isError(errorCode)) return errorCode; + op += errorCode; + + /* Compress */ + errorCode = FSE_buildCTable (&CTable, norm, maxSymbolValue, tableLog); + if (FSE_isError(errorCode)) return errorCode; + op += FSE_compress_usingCTable(op, oend - op, ip, srcSize, &CTable); + + /* check compressibility */ + if ( (size_t)(op-ostart) >= srcSize-1 ) + return 0; + + return op-ostart; +} + + +size_t FSE_compress (void* dst, size_t dstSize, const void* src, size_t srcSize) +{ + return FSE_compress2(dst, dstSize, src, (U32)srcSize, FSE_MAX_SYMBOL_VALUE, FSE_DEFAULT_TABLELOG); +} + + +/********************************************************* +* Decompression (Byte symbols) +*********************************************************/ +typedef struct +{ + U16 newState; + BYTE symbol; + BYTE nbBits; +} FSE_decode_t; /* size == U32 */ + +/* Specific corner case : RLE compression */ +size_t FSE_decompressRLE(void* dst, size_t originalSize, + const void* cSrc, size_t cSrcSize) +{ + if (cSrcSize != 1) return (size_t)-FSE_ERROR_srcSize_wrong; + memset(dst, *(BYTE*)cSrc, originalSize); + return originalSize; +} + + +size_t FSE_buildDTable_rle (void* DTable, BYTE symbolValue) +{ + U32* const base32 = (U32*)DTable; + FSE_decode_t* const cell = (FSE_decode_t*)(base32 + 1); + + /* Sanity check */ + if (((size_t)DTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */ + + base32[0] = 0; + + cell->newState = 0; + cell->symbol = symbolValue; + cell->nbBits = 0; + + return 0; +} + + +size_t FSE_buildDTable_raw (void* DTable, unsigned nbBits) +{ + U32* const base32 = (U32*)DTable; + FSE_decode_t* dinfo = (FSE_decode_t*)(base32 + 1); + const unsigned tableSize = 1 << nbBits; + const unsigned tableMask = tableSize - 1; + const unsigned maxSymbolValue = tableMask; + unsigned s; + + /* Sanity checks */ + if (nbBits < 1) return (size_t)-FSE_ERROR_GENERIC; /* min size */ + if (((size_t)DTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */ + + /* Build Decoding Table */ + base32[0] = nbBits; + for (s=0; s<=maxSymbolValue; s++) + { + dinfo[s].newState = 0; + dinfo[s].symbol = (BYTE)s; + dinfo[s].nbBits = (BYTE)nbBits; + } + + return 0; +} + + +/* FSE_initDStream + * Initialize a FSE_DStream_t. + * srcBuffer must point at the beginning of an FSE block. + * The function result is the size of the FSE_block (== srcSize). + * If srcSize is too small, the function will return an errorCode; + */ +size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSize) +{ + if (srcSize < 1) return (size_t)-FSE_ERROR_srcSize_wrong; + + if (srcSize >= sizeof(bitD_t)) + { + U32 contain32; + bitD->start = (char*)srcBuffer; + bitD->ptr = (char*)srcBuffer + srcSize - sizeof(bitD_t); + bitD->bitContainer = FSE_readLEST(bitD->ptr); + contain32 = ((BYTE*)srcBuffer)[srcSize-1]; + if (contain32 == 0) return (size_t)-FSE_ERROR_GENERIC; /* stop bit not present */ + bitD->bitsConsumed = 8 - FSE_highbit32(contain32); + } + else + { + U32 contain32; + bitD->start = (char*)srcBuffer; + bitD->ptr = bitD->start; + bitD->bitContainer = *(BYTE*)(bitD->start); + switch(srcSize) + { + case 7: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[6]) << (sizeof(bitD_t)*8 - 16); + case 6: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[5]) << (sizeof(bitD_t)*8 - 24); + case 5: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[4]) << (sizeof(bitD_t)*8 - 32); + case 4: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[3]) << 24; + case 3: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[2]) << 16; + case 2: bitD->bitContainer += (bitD_t)(((BYTE*)(bitD->start))[1]) << 8; + default:; + } + contain32 = ((BYTE*)srcBuffer)[srcSize-1]; + if (contain32 == 0) return (size_t)-FSE_ERROR_GENERIC; /* stop bit not present */ + bitD->bitsConsumed = 8 - FSE_highbit32(contain32); + bitD->bitsConsumed += (U32)(sizeof(bitD_t) - srcSize)*8; + } + + return srcSize; +} + + +/* FSE_readBits + * Read next n bits from the bitContainer. + * Use the fast variant *only* if n > 0. + * Note : for this function to work properly on 32-bits, don't read more than maxNbBits==25 + * return : value extracted. + */ +bitD_t FSE_readBits(FSE_DStream_t* bitD, U32 nbBits) +{ + bitD_t value = ((bitD->bitContainer << bitD->bitsConsumed) >> 1) >> (((sizeof(bitD_t)*8)-1)-nbBits); + bitD->bitsConsumed += nbBits; + return value; +} + +bitD_t FSE_readBitsFast(FSE_DStream_t* bitD, U32 nbBits) /* only if nbBits >= 1 */ +{ + bitD_t value = (bitD->bitContainer << bitD->bitsConsumed) >> ((sizeof(bitD_t)*8)-nbBits); + bitD->bitsConsumed += nbBits; + return value; +} + +unsigned FSE_reloadDStream(FSE_DStream_t* bitD) +{ + if (bitD->ptr >= bitD->start + sizeof(bitD_t)) + { + bitD->ptr -= bitD->bitsConsumed >> 3; + bitD->bitsConsumed &= 7; + bitD->bitContainer = FSE_readLEST(bitD->ptr); + return 0; + } + if (bitD->ptr == bitD->start) + { + if (bitD->bitsConsumed < sizeof(bitD_t)*8) return 1; + if (bitD->bitsConsumed == sizeof(bitD_t)*8) return 2; + return 3; + } + { + U32 nbBytes = bitD->bitsConsumed >> 3; + if (bitD->ptr - nbBytes < bitD->start) + nbBytes = (U32)(bitD->ptr - bitD->start); /* note : necessarily ptr > start */ + bitD->ptr -= nbBytes; + bitD->bitsConsumed -= nbBytes*8; + bitD->bitContainer = FSE_readLEST(bitD->ptr); /* note : necessarily srcSize > sizeof(bitD) */ + return (bitD->ptr == bitD->start); + } +} + + +void FSE_initDState(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD, const void* DTable) +{ + const U32* const base32 = (const U32*)DTable; + DStatePtr->state = FSE_readBits(bitD, base32[0]); + FSE_reloadDStream(bitD); + DStatePtr->table = base32 + 1; +} + +BYTE FSE_decodeSymbol(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD) +{ + const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; + const U32 nbBits = DInfo.nbBits; + BYTE symbol = DInfo.symbol; + bitD_t lowBits = FSE_readBits(bitD, nbBits); + + DStatePtr->state = DInfo.newState + lowBits; + return symbol; +} + +BYTE FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD) +{ + const FSE_decode_t DInfo = ((const FSE_decode_t*)(DStatePtr->table))[DStatePtr->state]; + const U32 nbBits = DInfo.nbBits; + BYTE symbol = DInfo.symbol; + bitD_t lowBits = FSE_readBitsFast(bitD, nbBits); + + DStatePtr->state = DInfo.newState + lowBits; + return symbol; +} + +/* FSE_endOfDStream + Tells if bitD has reached end of bitStream or not */ + +unsigned FSE_endOfDStream(const FSE_DStream_t* bitD) +{ + return FSE_reloadDStream((FSE_DStream_t*)bitD)==2; +} + +unsigned FSE_endOfDState(const FSE_DState_t* statePtr) +{ + return statePtr->state == 0; +} + + +FORCE_INLINE size_t FSE_decompress_usingDTable_generic( + void* dst, size_t maxDstSize, + const void* cSrc, size_t cSrcSize, + const void* DTable, unsigned fast) +{ + BYTE* const ostart = (BYTE*) dst; + BYTE* op = ostart; + BYTE* const omax = op + maxDstSize; + BYTE* const olimit = omax-3; + + FSE_DStream_t bitD; + FSE_DState_t state1, state2; + size_t errorCode; + + /* Init */ + errorCode = FSE_initDStream(&bitD, cSrc, cSrcSize); /* replaced last arg by maxCompressed Size */ + if (FSE_isError(errorCode)) return errorCode; + + FSE_initDState(&state1, &bitD, DTable); + FSE_initDState(&state2, &bitD, DTable); + + + /* 2 symbols per loop */ + while (!FSE_reloadDStream(&bitD) && (op sizeof(bitD_t)*8) /* This test must be static */ + FSE_reloadDStream(&bitD); + + *op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD); + + if (FSE_MAX_TABLELOG*4+7 < sizeof(bitD_t)*8) /* This test must be static */ + { + *op++ = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD); + *op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD); + } + } + + /* tail */ + while (1) + { + if ( (FSE_reloadDStream(&bitD)>2) || (op==omax) || (FSE_endOfDState(&state1) && FSE_endOfDStream(&bitD)) ) + break; + + *op++ = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD); + + if ( (FSE_reloadDStream(&bitD)>2) || (op==omax) || (FSE_endOfDState(&state2) && FSE_endOfDStream(&bitD)) ) + break; + + *op++ = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD); + } + + /* end ? */ + if (FSE_endOfDStream(&bitD) && FSE_endOfDState(&state1) && FSE_endOfDState(&state2) ) + return op-ostart; + + if (op==omax) return (size_t)-FSE_ERROR_dstSize_tooSmall; /* dst buffer is full, but cSrc unfinished */ + + return (size_t)-FSE_ERROR_corruptionDetected; +} + + +size_t FSE_decompress_usingDTable(void* dst, size_t originalSize, + const void* cSrc, size_t cSrcSize, + const void* DTable, size_t fastMode) +{ + /* select fast mode (static) */ + if (fastMode) return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, DTable, 1); + return FSE_decompress_usingDTable_generic(dst, originalSize, cSrc, cSrcSize, DTable, 0); +} + + +size_t FSE_decompress(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize) +{ + const BYTE* const istart = (const BYTE*)cSrc; + const BYTE* ip = istart; + short counting[FSE_MAX_SYMBOL_VALUE+1]; + FSE_decode_t DTable[FSE_DTABLE_SIZE_U32(FSE_MAX_TABLELOG)]; + unsigned maxSymbolValue = FSE_MAX_SYMBOL_VALUE; + unsigned tableLog; + size_t errorCode, fastMode; + + if (cSrcSize<2) return (size_t)-FSE_ERROR_srcSize_wrong; /* too small input size */ + + /* normal FSE decoding mode */ + errorCode = FSE_readHeader (counting, &maxSymbolValue, &tableLog, istart, cSrcSize); + if (FSE_isError(errorCode)) return errorCode; + if (errorCode >= cSrcSize) return (size_t)-FSE_ERROR_srcSize_wrong; /* too small input size */ + ip += errorCode; + cSrcSize -= errorCode; + + fastMode = FSE_buildDTable (DTable, counting, maxSymbolValue, tableLog); + if (FSE_isError(fastMode)) return fastMode; + + /* always return, even if it is an error code */ + return FSE_decompress_usingDTable (dst, maxDstSize, ip, cSrcSize, DTable, fastMode); +} + + +#endif /* FSE_COMMONDEFS_ONLY */ + +/* + 2nd part of the file + designed to be included + for type-specific functions (template equivalent in C) + Objective is to write such functions only once, for better maintenance +*/ + +/* safety checks */ +#ifndef FSE_FUNCTION_EXTENSION +# error "FSE_FUNCTION_EXTENSION must be defined" +#endif +#ifndef FSE_FUNCTION_TYPE +# error "FSE_FUNCTION_TYPE must be defined" +#endif + +/* Function names */ +#define FSE_CAT(X,Y) X##Y +#define FSE_FUNCTION_NAME(X,Y) FSE_CAT(X,Y) +#define FSE_TYPE_NAME(X,Y) FSE_CAT(X,Y) + + +/* Function templates */ +size_t FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (unsigned* count, const FSE_FUNCTION_TYPE* source, size_t sourceSize, unsigned* maxSymbolValuePtr, unsigned safe) +{ + const FSE_FUNCTION_TYPE* ip = source; + const FSE_FUNCTION_TYPE* const iend = ip+sourceSize; + unsigned maxSymbolValue = *maxSymbolValuePtr; + unsigned max=0; + int s; + + U32 Counting1[FSE_MAX_SYMBOL_VALUE+1] = { 0 }; + U32 Counting2[FSE_MAX_SYMBOL_VALUE+1] = { 0 }; + U32 Counting3[FSE_MAX_SYMBOL_VALUE+1] = { 0 }; + U32 Counting4[FSE_MAX_SYMBOL_VALUE+1] = { 0 }; + + /* safety checks */ + if (!sourceSize) + { + memset(count, 0, (maxSymbolValue + 1) * sizeof(FSE_FUNCTION_TYPE)); + *maxSymbolValuePtr = 0; + return 0; + } + if (maxSymbolValue > FSE_MAX_SYMBOL_VALUE) return (size_t)-FSE_ERROR_GENERIC; /* maxSymbolValue too large : unsupported */ + if (!maxSymbolValue) maxSymbolValue = FSE_MAX_SYMBOL_VALUE; /* 0 == default */ + + if ((safe) || (sizeof(FSE_FUNCTION_TYPE)>1)) + { + /* check input values, to avoid count table overflow */ + while (ip < iend-3) + { + if (*ip>maxSymbolValue) return (size_t)-FSE_ERROR_GENERIC; Counting1[*ip++]++; + if (*ip>maxSymbolValue) return (size_t)-FSE_ERROR_GENERIC; Counting2[*ip++]++; + if (*ip>maxSymbolValue) return (size_t)-FSE_ERROR_GENERIC; Counting3[*ip++]++; + if (*ip>maxSymbolValue) return (size_t)-FSE_ERROR_GENERIC; Counting4[*ip++]++; + } + } + else + { + U32 cached = FSE_read32(ip); ip += 4; + while (ip < iend-15) + { + U32 c = cached; cached = FSE_read32(ip); ip += 4; + Counting1[(BYTE) c ]++; + Counting2[(BYTE)(c>>8) ]++; + Counting3[(BYTE)(c>>16)]++; + Counting4[ c>>24 ]++; + c = cached; cached = FSE_read32(ip); ip += 4; + Counting1[(BYTE) c ]++; + Counting2[(BYTE)(c>>8) ]++; + Counting3[(BYTE)(c>>16)]++; + Counting4[ c>>24 ]++; + c = cached; cached = FSE_read32(ip); ip += 4; + Counting1[(BYTE) c ]++; + Counting2[(BYTE)(c>>8) ]++; + Counting3[(BYTE)(c>>16)]++; + Counting4[ c>>24 ]++; + c = cached; cached = FSE_read32(ip); ip += 4; + Counting1[(BYTE) c ]++; + Counting2[(BYTE)(c>>8) ]++; + Counting3[(BYTE)(c>>16)]++; + Counting4[ c>>24 ]++; + } + ip-=4; + } + + /* finish last symbols */ + while (ipmaxSymbolValue)) return (size_t)-FSE_ERROR_GENERIC; Counting1[*ip++]++; } + + for (s=0; s<=(int)maxSymbolValue; s++) + { + count[s] = Counting1[s] + Counting2[s] + Counting3[s] + Counting4[s]; + if (count[s] > max) max = count[s]; + } + + while (!count[maxSymbolValue]) maxSymbolValue--; + *maxSymbolValuePtr = maxSymbolValue; + return (int)max; +} + +/* hidden fast variant (unsafe) */ +size_t FSE_FUNCTION_NAME(FSE_countFast, FSE_FUNCTION_EXTENSION) (unsigned* count, const FSE_FUNCTION_TYPE* source, size_t sourceSize, unsigned* maxSymbolValuePtr) +{ + return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, source, sourceSize, maxSymbolValuePtr, 0); +} + +size_t FSE_FUNCTION_NAME(FSE_count, FSE_FUNCTION_EXTENSION) (unsigned* count, const FSE_FUNCTION_TYPE* source, size_t sourceSize, unsigned* maxSymbolValuePtr) +{ + if ((sizeof(FSE_FUNCTION_TYPE)==1) && (*maxSymbolValuePtr >= 255)) + { + *maxSymbolValuePtr = 255; + return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, source, sourceSize, maxSymbolValuePtr, 0); + } + return FSE_FUNCTION_NAME(FSE_count_generic, FSE_FUNCTION_EXTENSION) (count, source, sourceSize, maxSymbolValuePtr, 1); +} + + +static U32 FSE_tableStep(U32 tableSize) { return (tableSize>>1) + (tableSize>>3) + 3; } + +size_t FSE_FUNCTION_NAME(FSE_buildCTable, FSE_FUNCTION_EXTENSION) +(void* CTable, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog) +{ + const unsigned tableSize = 1 << tableLog; + const unsigned tableMask = tableSize - 1; + U16* tableU16 = ( (U16*) CTable) + 2; + FSE_symbolCompressionTransform* symbolTT = (FSE_symbolCompressionTransform*) (((U32*)CTable) + 1 + (tableLog ? tableSize>>1 : 1) ); + const unsigned step = FSE_tableStep(tableSize); + unsigned cumul[FSE_MAX_SYMBOL_VALUE+2]; + U32 position = 0; + FSE_FUNCTION_TYPE tableSymbol[FSE_MAX_TABLESIZE]; + U32 highThreshold = tableSize-1; + unsigned symbol; + unsigned i; + + /* safety checks */ + if (((size_t)CTable) & 3) return (size_t)-FSE_ERROR_GENERIC; /* Must be allocated of 4 bytes boundaries */ + + /* header */ + tableU16[-2] = (U16) tableLog; + tableU16[-1] = (U16) maxSymbolValue; + + /* For explanations on how to distribute symbol values over the table : + * http://fastcompression.blogspot.fr/2014/02/fse-distributing-symbol-values.html */ + + /* symbol start positions */ + cumul[0] = 0; + for (i=1; i<=maxSymbolValue+1; i++) + { + if (normalizedCounter[i-1]==-1) /* Low prob symbol */ + { + cumul[i] = cumul[i-1] + 1; + tableSymbol[highThreshold--] = (FSE_FUNCTION_TYPE)(i-1); + } + else + cumul[i] = cumul[i-1] + normalizedCounter[i-1]; + } + cumul[maxSymbolValue+1] = tableSize+1; + + /* Spread symbols */ + for (symbol=0; symbol<=maxSymbolValue; symbol++) + { + int nbOccurences; + for (nbOccurences=0; nbOccurences highThreshold) position = (position + step) & tableMask; /* Lowprob area */ + } + } + + if (position!=0) return (size_t)-FSE_ERROR_GENERIC; /* Must have gone through all positions */ + + /* Build table */ + for (i=0; i FSE_TABLELOG_ABSOLUTE_MAX) tableLog = FSE_TABLELOG_ABSOLUTE_MAX; + return malloc( ((size_t)1< FSE_MAX_SYMBOL_VALUE) return (size_t)-FSE_ERROR_maxSymbolValue_tooLarge; + if (tableLog > FSE_MAX_TABLELOG) return (size_t)-FSE_ERROR_tableLog_tooLarge; + + /* Init, lay down lowprob symbols */ + base32[0] = tableLog; + for (s=0; s<=maxSymbolValue; s++) + { + if (normalizedCounter[s]==-1) + { + tableDecode[highThreshold--].symbol = (FSE_FUNCTION_TYPE)s; + symbolNext[s] = 1; + } + else + { + if (normalizedCounter[s] >= largeLimit) noLarge=0; + symbolNext[s] = normalizedCounter[s]; + } + } + + /* Spread symbols */ + for (s=0; s<=maxSymbolValue; s++) + { + int i; + for (i=0; i highThreshold) position = (position + step) & tableMask; /* lowprob area */ + } + } + + if (position!=0) return (size_t)-FSE_ERROR_GENERIC; /* position must reach all cells once, otherwise normalizedCounter is incorrect */ + + /* Build Decoding table */ + { + U32 i; + for (i=0; i // size_t, ptrdiff_t + + +/****************************************** +* FSE simple functions +******************************************/ +size_t FSE_compress(void* dst, size_t maxDstSize, + const void* src, size_t srcSize); +size_t FSE_decompress(void* dst, size_t maxDstSize, + const void* cSrc, size_t cSrcSize); +/* +FSE_compress(): + Compress content of buffer 'src', of size 'srcSize', into destination buffer 'dst'. + 'dst' buffer must be already allocated, and sized to handle worst case situations. + Worst case size evaluation is provided by FSE_compressBound(). + return : size of compressed data + Special values : if result == 0, data is uncompressible => Nothing is stored within cSrc !! + if result == 1, data is one constant element x srcSize times. Use RLE compression. + if FSE_isError(result), it's an error code. + +FSE_decompress(): + Decompress FSE data from buffer 'cSrc', of size 'cSrcSize', + into already allocated destination buffer 'dst', of size 'maxDstSize'. + ** Important ** : This function doesn't decompress uncompressed nor RLE data ! + return : size of regenerated data (<= maxDstSize) + or an error code, which can be tested using FSE_isError() +*/ + + +size_t FSE_decompressRLE(void* dst, size_t originalSize, + const void* cSrc, size_t cSrcSize); +/* +FSE_decompressRLE(): + Decompress specific RLE corner case (equivalent to memset()). + cSrcSize must be == 1. originalSize must be exact. + return : size of regenerated data (==originalSize) + or an error code, which can be tested using FSE_isError() + +Note : there is no function provided for uncompressed data, as it's just a simple memcpy() +*/ + + +/****************************************** +* Tool functions +******************************************/ +size_t FSE_compressBound(size_t size); /* maximum compressed size */ + +/* Error Management */ +unsigned FSE_isError(size_t code); /* tells if a return value is an error code */ +const char* FSE_getErrorName(size_t code); /* provides error code string (useful for debugging) */ + + +/****************************************** +* FSE advanced functions +******************************************/ +/* +FSE_compress2(): + Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog' + Both parameters can be defined as '0' to mean : use default value + return : size of compressed data + or -1 if there is an error +*/ +size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog); + + +/****************************************** + FSE detailed API +******************************************/ +/* +int FSE_compress(char* dest, const char* source, int inputSize) does the following: +1. count symbol occurrence from source[] into table count[] +2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog) +3. save normalized counters to memory buffer using writeHeader() +4. build encoding table 'CTable' from normalized counters +5. encode the data stream using encoding table + +int FSE_decompress(char* dest, int originalSize, const char* compressed) performs: +1. read normalized counters with readHeader() +2. build decoding table 'DTable' from normalized counters +3. decode the data stream using decoding table + +The following API allows triggering specific sub-functions. +*/ + +/* *** COMPRESSION *** */ + +size_t FSE_count(unsigned* count, const unsigned char* src, size_t srcSize, unsigned* maxSymbolValuePtr); + +unsigned FSE_optimalTableLog(unsigned tableLog, size_t srcSize, unsigned maxSymbolValue); +size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog, const unsigned* count, size_t total, unsigned maxSymbolValue); + +size_t FSE_headerBound(unsigned maxSymbolValue, unsigned tableLog); +size_t FSE_writeHeader (void* headerBuffer, size_t headerBufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); + +void* FSE_createCTable (unsigned tableLog, unsigned maxSymbolValue); +void FSE_freeCTable (void* CTable); +size_t FSE_buildCTable(void* CTable, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); + +size_t FSE_compress_usingCTable (void* dst, size_t dstSize, const void* src, size_t srcSize, const void* CTable); + +/* +The first step is to count all symbols. FSE_count() provides one quick way to do this job. +Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have '*maxSymbolValuePtr+1' cells. +'source' is a table of char of size 'sourceSize'. All values within 'src' MUST be <= *maxSymbolValuePtr +*maxSymbolValuePtr will be updated, with its real value (necessarily <= original value) +FSE_count() will return the number of occurrence of the most frequent symbol. +If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()). + +The next step is to normalize the frequencies. +FSE_normalizeCount() will ensure that sum of frequencies is == 2 ^'tableLog'. +It also guarantees a minimum of 1 to any Symbol which frequency is >= 1. +You can use input 'tableLog'==0 to mean "use default tableLog value". +If you are unsure of which tableLog value to use, you can optionally call FSE_optimalTableLog(), +which will provide the optimal valid tableLog given sourceSize, maxSymbolValue, and a user-defined maximum (0 means "default"). + +The result of FSE_normalizeCount() will be saved into a table, +called 'normalizedCounter', which is a table of signed short. +'normalizedCounter' must be already allocated, and have at least 'maxSymbolValue+1' cells. +The return value is tableLog if everything proceeded as expected. +It is 0 if there is a single symbol within distribution. +If there is an error(typically, invalid tableLog value), the function will return an ErrorCode (which can be tested using FSE_isError()). + +'normalizedCounter' can be saved in a compact manner to a memory area using FSE_writeHeader(). +'header' buffer must be already allocated. +For guaranteed success, buffer size must be at least FSE_headerBound(). +The result of the function is the number of bytes written into 'header'. +If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()) (for example, buffer size too small). + +'normalizedCounter' can then be used to create the compression tables 'CTable'. +The space required by 'CTable' must be already allocated. Its size is provided by FSE_sizeof_CTable(). +'CTable' must be aligned of 4 bytes boundaries. +You can then use FSE_buildCTable() to fill 'CTable'. +In both cases, if there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()). + +'CTable' can then be used to compress 'source', with FSE_compress_usingCTable(). +Similar to FSE_count(), the convention is that 'source' is assumed to be a table of char of size 'sourceSize' +The function returns the size of compressed data (without header), or -1 if failed. +*/ + + +/* *** DECOMPRESSION *** */ + +size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* headerBuffer, size_t hbSize); + +void* FSE_createDTable(unsigned tableLog); +void FSE_freeDTable(void* DTable); +size_t FSE_buildDTable (void* DTable, const short* const normalizedCounter, unsigned maxSymbolValue, unsigned tableLog); + +size_t FSE_decompress_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const void* DTable, size_t fastMode); + +/* +If the block is RLE compressed, or uncompressed, use the relevant specific functions. + +The first step is to obtain the normalized frequencies of symbols. +This can be performed by reading a header with FSE_readHeader(). +'normalizedCounter' must be already allocated, and have at least '*maxSymbolValuePtr+1' cells of short. +In practice, that means it's necessary to know 'maxSymbolValue' beforehand, +or size the table to handle worst case situations (typically 256). +FSE_readHeader will provide 'tableLog' and 'maxSymbolValue' stored into the header. +The result of FSE_readHeader() is the number of bytes read from 'header'. +The following values have special meaning : +return 2 : there is only a single symbol value. The value is provided into the second byte of header. +return 1 : data is uncompressed +If there is an error, the function will return an error code, which can be tested using FSE_isError(). + +The next step is to create the decompression tables 'DTable' from 'normalizedCounter'. +This is performed by the function FSE_buildDTable(). +The space required by 'DTable' must be already allocated and properly aligned. +One can create a DTable using FSE_createDTable(). +The function will return 1 if DTable is compatible with fastMode, 0 otherwise. +If there is an error, the function will return an error code, which can be tested using FSE_isError(). + +'DTable' can then be used to decompress 'compressed', with FSE_decompress_usingDTable(). +Only trigger fastMode if it was authorized by result of FSE_buildDTable(), otherwise decompression will fail. +cSrcSize must be correct, otherwise decompression will fail. +FSE_decompress_usingDTable() result will tell how many bytes were regenerated. +If there is an error, the function will return an error code, which can be tested using FSE_isError(). +*/ + + +/****************************************** +* FSE streaming compression API +******************************************/ +typedef struct +{ + size_t bitContainer; + int bitPos; + char* startPtr; + char* ptr; +} FSE_CStream_t; + +typedef struct +{ + ptrdiff_t value; + const void* stateTable; + const void* symbolTT; + unsigned stateLog; +} FSE_CState_t; + +void FSE_initCStream(FSE_CStream_t* bitC, void* dstBuffer); +void FSE_initCState(FSE_CState_t* CStatePtr, const void* CTable); + +void FSE_encodeByte(FSE_CStream_t* bitC, FSE_CState_t* CStatePtr, unsigned char symbol); +void FSE_addBits(FSE_CStream_t* bitC, size_t value, unsigned nbBits); +void FSE_flushBits(FSE_CStream_t* bitC); + +void FSE_flushCState(FSE_CStream_t* bitC, const FSE_CState_t* CStatePtr); +size_t FSE_closeCStream(FSE_CStream_t* bitC); + +/* +These functions are inner components of FSE_compress_usingCTable(). +They allow creation of custom streams, mixing multiple tables and bit sources. + +A key property to keep in mind is that encoding and decoding are done **in reverse direction**. +So the first symbol you will encode is the last you will decode, like a lifo stack. + +You will need a few variables to track your CStream. They are : + +void* CTable; // Provided by FSE_buildCTable() +FSE_CStream_t bitC; // bitStream tracking structure +FSE_CState_t state; // State tracking structure + + +The first thing to do is to init the bitStream, and the state. + FSE_initCStream(&bitC, dstBuffer); + FSE_initState(&state, CTable); + +You can then encode your input data, byte after byte. +FSE_encodeByte() outputs a maximum of 'tableLog' bits at a time. +Remember decoding will be done in reverse direction. + FSE_encodeByte(&bitStream, &state, symbol); + +At any time, you can add any bit sequence. +Note : maximum allowed nbBits is 25, for compatibility with 32-bits decoders + FSE_addBits(&bitStream, bitField, nbBits); + +The above methods don't commit data to memory, they just store it into local register, for speed. +Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t). +Writing data to memory is a manual operation, performed by the flushBits function. + FSE_flushBits(&bitStream); + +Your last FSE encoding operation shall be to flush your last state value(s). + FSE_flushState(&bitStream, &state); + +You must then close the bitStream if you opened it with FSE_initCStream(). +It's possible to embed some user-info into the header, as an optionalId [0-31]. +The function returns the size in bytes of CStream. +If there is an error, it returns an errorCode (which can be tested using FSE_isError()). + size_t size = FSE_closeCStream(&bitStream, optionalId); +*/ + + +/****************************************** +* FSE streaming decompression API +******************************************/ +//typedef unsigned int bitD_t; +typedef size_t bitD_t; + +typedef struct +{ + bitD_t bitContainer; + unsigned bitsConsumed; + const char* ptr; + const char* start; +} FSE_DStream_t; + +typedef struct +{ + bitD_t state; + const void* table; +} FSE_DState_t; + + +size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSize); +void FSE_initDState(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD, const void* DTable); + +unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD); +bitD_t FSE_readBits(FSE_DStream_t* bitD, unsigned nbBits); +unsigned int FSE_reloadDStream(FSE_DStream_t* bitD); + +unsigned FSE_endOfDStream(const FSE_DStream_t* bitD); +unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr); + +/* +Let's now decompose FSE_decompress_usingDTable() into its unitary elements. +You will decode FSE-encoded symbols from the bitStream, +and also any other bitFields you put in, **in reverse order**. + +You will need a few variables to track your bitStream. They are : + +FSE_DStream_t DStream; // Stream context +FSE_DState_t DState; // State context. Multiple ones are possible +const void* DTable; // Decoding table, provided by FSE_buildDTable() +U32 tableLog; // Provided by FSE_readHeader() + +The first thing to do is to init the bitStream. + errorCode = FSE_initDStream(&DStream, &optionalId, srcBuffer, srcSize); + +You should then retrieve your initial state(s) (multiple ones are possible) : + errorCode = FSE_initDState(&DState, &DStream, DTable, tableLog); + +You can then decode your data, symbol after symbol. +For information the maximum number of bits read by FSE_decodeSymbol() is 'tableLog'. +Keep in mind that symbols are decoded in reverse order, like a lifo stack (last in, first out). + unsigned char symbol = FSE_decodeSymbol(&DState, &DStream); + +You can retrieve any bitfield you eventually stored into the bitStream (in reverse order) +Note : maximum allowed nbBits is 25 + unsigned int bitField = FSE_readBits(&DStream, nbBits); + +All above operations only read from local register (which size is controlled by bitD_t==32 bits). +Reading data from memory is manually performed by the reload method. + endSignal = FSE_reloadDStream(&DStream); + +FSE_reloadDStream() result tells if there is still some more data to read from DStream. +0 : there is still some data left into the DStream. +1 Dstream reached end of buffer, but is not yet fully extracted. It will not load data from memory any more. +2 Dstream reached its exact end, corresponding in general to decompression completed. +3 Dstream went too far. Decompression result is corrupted. + +When reaching end of buffer(1), progress slowly if you decode multiple symbols per loop, +to properly detect the exact end of stream. +After each decoded symbol, check if DStream is fully consumed using this simple test : + FSE_reloadDStream(&DStream) >= 2 + +When it's done, verify decompression is fully completed, by checking both DStream and the relevant states. +Checking if DStream has reached its end is performed by : + FSE_endOfDStream(&DStream); +Check also the states. There might be some entropy left there, still able to decode some high probability symbol. + FSE_endOfDState(&DState); +*/ + + +#if defined (__cplusplus) +} +#endif diff --git a/libs/libzstd/include/zstd/fse_static.h b/libs/libzstd/include/zstd/fse_static.h new file mode 100644 index 00000000000..7d400a5c117 --- /dev/null +++ b/libs/libzstd/include/zstd/fse_static.h @@ -0,0 +1,108 @@ +/* ****************************************************************** + FSE : Finite State Entropy coder + header file for static linking (only) + Copyright (C) 2013-2015, Yann Collet + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - Source repository : https://github.com/Cyan4973/FiniteStateEntropy + - Public forum : https://groups.google.com/forum/#!forum/lz4c +****************************************************************** */ +#pragma once + +#if defined (__cplusplus) +extern "C" { +#endif + + +/****************************************** +* Tool functions +******************************************/ +#define FSE_MAX_HEADERSIZE 512 +#define FSE_COMPRESSBOUND(size) (size + (size>>7) + FSE_MAX_HEADERSIZE) /* Macro can be useful for static allocation */ + + +/****************************************** +* Static allocation +******************************************/ +/* You can statically allocate a CTable as a table of U32 using below macro */ +#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2)) +#define FSE_DTABLE_SIZE_U32(maxTableLog) ((1<= 1 (otherwise, result will be corrupted) */ + +unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD); +/* faster, but works only if nbBits >= 1 (otherwise, result will be corrupted) */ + + +#if defined (__cplusplus) +} +#endif diff --git a/libs/libzstd/include/zstd/zstd.c b/libs/libzstd/include/zstd/zstd.c new file mode 100644 index 00000000000..c8eed5a3ea2 --- /dev/null +++ b/libs/libzstd/include/zstd/zstd.c @@ -0,0 +1,1810 @@ +/* + zstd - standard compression library + Copyright (C) 2014-2015, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - zstd source repository : https://github.com/Cyan4973/zstd + - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c +*/ + +/**************************************************************** +* Tuning parameters +*****************************************************************/ +/* MEMORY_USAGE : +* Memory usage formula : N->2^N Bytes (examples : 10 -> 1KB; 12 -> 4KB ; 16 -> 64KB; 20 -> 1MB; etc.) +* Increasing memory usage improves compression ratio +* Reduced memory usage can improve speed, due to cache effect */ +#define ZSTD_MEMORY_USAGE 17 + + +/************************************** + CPU Feature Detection +**************************************/ +/* + * Automated efficient unaligned memory access detection + * Based on known hardware architectures + * This list will be updated thanks to feedbacks + */ +#if defined(CPU_HAS_EFFICIENT_UNALIGNED_MEMORY_ACCESS) \ + || defined(__ARM_FEATURE_UNALIGNED) \ + || defined(__i386__) || defined(__x86_64__) \ + || defined(_M_IX86) || defined(_M_X64) \ + || defined(__ARM_ARCH_7__) || defined(__ARM_ARCH_8__) \ + || (defined(_M_ARM) && (_M_ARM >= 7)) +# define ZSTD_UNALIGNED_ACCESS 1 +#else +# define ZSTD_UNALIGNED_ACCESS 0 +#endif + + +/******************************************************** +* Includes +*********************************************************/ +#include /* calloc */ +#include /* memcpy, memmove */ +#include /* debug : printf */ +#include "zstd_static.h" +#if defined(__clang__) || defined(__GNUC__) +# include "fse.c" /* due to GCC/Clang inlining limitations, including *.c runs noticeably faster */ +#else +# include "fse_static.h" +#endif + + +/******************************************************** +* Compiler specifics +*********************************************************/ +#ifdef __AVX2__ +# include /* AVX2 intrinsics */ +#endif + +#ifdef _MSC_VER /* Visual Studio */ +# define FORCE_INLINE static __forceinline +# include /* For Visual 2005 */ +# pragma warning(disable : 4127) /* disable: C4127: conditional expression is constant */ +# pragma warning(disable : 4324) /* disable: C4324: padded structure */ +#else +# define GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) +# ifdef __GNUC__ +# define FORCE_INLINE static inline __attribute__((always_inline)) +# else +# define FORCE_INLINE static inline +# endif +#endif + + +#ifndef MEM_ACCESS_MODULE +#define MEM_ACCESS_MODULE +/******************************************************** +* Basic Types +*********************************************************/ +#if defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L /* C99 */ +# include +typedef uint8_t BYTE; +typedef uint16_t U16; +typedef int16_t S16; +typedef uint32_t U32; +typedef int32_t S32; +typedef uint64_t U64; +#else +typedef unsigned char BYTE; +typedef unsigned short U16; +typedef signed short S16; +typedef unsigned int U32; +typedef signed int S32; +typedef unsigned long long U64; +#endif + +#endif /* MEM_ACCESS_MODULE */ + + +/******************************************************** +* Constants +*********************************************************/ +static const U32 ZSTD_magicNumber = 0xFD2FB51C; /* Initial (limited) frame format */ + +#define HASH_LOG (ZSTD_MEMORY_USAGE - 2) +#define HASH_TABLESIZE (1 << HASH_LOG) +#define HASH_MASK (HASH_TABLESIZE - 1) + +#define KNUTH 2654435761 + +#define BIT7 128 +#define BIT6 64 +#define BIT5 32 +#define BIT4 16 + +#define KB *(1 <<10) +#define MB *(1 <<20) +#define GB *(1U<<30) + +#define BLOCKSIZE (128 KB) /* define, for static allocation */ +static const U32 g_maxDistance = 4 * BLOCKSIZE; +static const U32 g_maxLimit = 1 GB; +static const U32 g_searchStrength = 8; + +#define WORKPLACESIZE (BLOCKSIZE*11/4) +#define MINMATCH 4 +#define MLbits 7 +#define LLbits 6 +#define Offbits 5 +#define MaxML ((1<>8); + p[2] = (BYTE)(val32>>16); + p[3] = (BYTE)(val32>>24); + } +} + +static U32 ZSTD_readBE32(const void* memPtr) +{ + const BYTE* p = (const BYTE*)memPtr; + return (U32)(((U32)p[0]<<24) + ((U32)p[1]<<16) + ((U32)p[2]<<8) + ((U32)p[3]<<0)); +} + +static void ZSTD_writeBE32(void* memPtr, U32 value) +{ + BYTE* const p = (BYTE* const) memPtr; + p[0] = (BYTE)(value>>24); + p[1] = (BYTE)(value>>16); + p[2] = (BYTE)(value>>8); + p[3] = (BYTE)(value>>0); +} + +static size_t ZSTD_writeProgressive(void* ptr, size_t value) +{ + BYTE* const bStart = (BYTE* const)ptr; + BYTE* byte = bStart; + + do + { + BYTE l = value & 127; + value >>= 7; + if (value) l += 128; + *byte++ = l; + } while (value); + + return byte - bStart; +} + + +static size_t ZSTD_readProgressive(size_t* result, const void* ptr) +{ + const BYTE* const bStart = (const BYTE* const)ptr; + const BYTE* byte = bStart; + size_t r = 0; + U32 shift = 0; + + do + { + r += (*byte & 127) << shift; + shift += 7; + } while (*byte++ & 128); + + *result = r; + return byte - bStart; +} + + +/************************************** +* Local structures +***************************************/ +typedef enum { bt_compressed, bt_raw, bt_rle, bt_end } blockType_t; + +typedef struct +{ + blockType_t blockType; + U32 origSize; +} blockProperties_t; + +typedef struct { + void* buffer; + U32* offsetStart; + U32* offset; + BYTE* litStart; + BYTE* lit; + BYTE* litLengthStart; + BYTE* litLength; + BYTE* matchLengthStart; + BYTE* matchLength; + BYTE* dumpsStart; + BYTE* dumps; +} seqStore_t; + +void ZSTD_resetSeqStore(seqStore_t* ssPtr) +{ + ssPtr->offset = ssPtr->offsetStart; + ssPtr->lit = ssPtr->litStart; + ssPtr->litLength = ssPtr->litLengthStart; + ssPtr->matchLength = ssPtr->matchLengthStart; + ssPtr->dumps = ssPtr->dumpsStart; +} + + +typedef struct +{ + const BYTE* base; + U32 current; + U32 nextUpdate; + seqStore_t seqStore; +#ifdef __AVX2__ + __m256i hashTable[HASH_TABLESIZE>>3]; +#else + U32 hashTable[HASH_TABLESIZE]; +#endif +} cctxi_t; + + +ZSTD_cctx_t ZSTD_createCCtx(void) +{ + cctxi_t* ctx = (cctxi_t*) malloc( sizeof(cctxi_t) ); + ctx->seqStore.buffer = malloc(WORKPLACESIZE); + ctx->seqStore.offsetStart = (U32*) (ctx->seqStore.buffer); + ctx->seqStore.litStart = (BYTE*) (ctx->seqStore.offsetStart + (BLOCKSIZE>>2)); + ctx->seqStore.litLengthStart = ctx->seqStore.litStart + BLOCKSIZE; + ctx->seqStore.matchLengthStart = ctx->seqStore.litLengthStart + (BLOCKSIZE>>2); + ctx->seqStore.dumpsStart = ctx->seqStore.matchLengthStart + (BLOCKSIZE>>2); + return (ZSTD_cctx_t)ctx; +} + +void ZSTD_resetCCtx(ZSTD_cctx_t cctx) +{ + cctxi_t* ctx = (cctxi_t*)cctx; + ctx->base = NULL; + memset(ctx->hashTable, 0, HASH_TABLESIZE*4); +} + +size_t ZSTD_freeCCtx(ZSTD_cctx_t cctx) +{ + cctxi_t* ctx = (cctxi_t*) (cctx); + free(ctx->seqStore.buffer); + free(ctx); + return 0; +} + + +/************************************** +* Error Management +**************************************/ +/* tells if a return value is an error code */ +unsigned ZSTD_isError(size_t code) +{ + return (code > (size_t)(-ZSTD_ERROR_maxCode)); +} + +#define ZSTD_GENERATE_STRING(STRING) #STRING, +static const char* ZSTD_errorStrings[] = { ZSTD_LIST_ERRORS(ZSTD_GENERATE_STRING) }; + +/* provides error code string (useful for debugging) */ +const char* ZSTD_getErrorName(size_t code) +{ + static const char* codeError = "Unspecified error code"; + if (ZSTD_isError(code)) return ZSTD_errorStrings[-(int)(code)]; + return codeError; +} + + +/************************************** +* Tool functions +**************************************/ +unsigned ZSTD_versionNumber (void) { return ZSTD_VERSION_NUMBER; } + +static unsigned ZSTD_highbit(U32 val) +{ +# if defined(_MSC_VER) /* Visual */ + unsigned long r; + _BitScanReverse(&r, val); + return (unsigned)r; +# elif defined(__GNUC__) && (GCC_VERSION >= 304) /* GCC Intrinsic */ + return 31 - __builtin_clz(val); +# else /* Software version */ + static const int DeBruijnClz[32] = { 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 }; + U32 v = val; + int r; + v |= v >> 1; + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + r = DeBruijnClz[(U32)(v * 0x07C4ACDDU) >> 27]; + return r; +# endif +} + +static unsigned ZSTD_NbCommonBytes (register size_t val) +{ + if (ZSTD_isLittleEndian()) + { + if (ZSTD_64bits()) + { +# if defined(_MSC_VER) && defined(_WIN64) && !defined(LZ4_FORCE_SW_BITCOUNT) + unsigned long r = 0; + _BitScanForward64( &r, (U64)val ); + return (int)(r>>3); +# elif defined(__GNUC__) && (GCC_VERSION >= 304) && !defined(LZ4_FORCE_SW_BITCOUNT) + return (__builtin_ctzll((U64)val) >> 3); +# else + static const int DeBruijnBytePos[64] = { 0, 0, 0, 0, 0, 1, 1, 2, 0, 3, 1, 3, 1, 4, 2, 7, 0, 2, 3, 6, 1, 5, 3, 5, 1, 3, 4, 4, 2, 5, 6, 7, 7, 0, 1, 2, 3, 3, 4, 6, 2, 6, 5, 5, 3, 4, 5, 6, 7, 1, 2, 4, 6, 4, 4, 5, 7, 2, 6, 5, 7, 6, 7, 7 }; + return DeBruijnBytePos[((U64)((val & -(long long)val) * 0x0218A392CDABBD3FULL)) >> 58]; +# endif + } + else /* 32 bits */ + { +# if defined(_MSC_VER) && !defined(LZ4_FORCE_SW_BITCOUNT) + unsigned long r; + _BitScanForward( &r, (U32)val ); + return (int)(r>>3); +# elif defined(__GNUC__) && (GCC_VERSION >= 304) && !defined(LZ4_FORCE_SW_BITCOUNT) + return (__builtin_ctz((U32)val) >> 3); +# else + static const int DeBruijnBytePos[32] = { 0, 0, 3, 0, 3, 1, 3, 0, 3, 2, 2, 1, 3, 2, 0, 1, 3, 3, 1, 2, 2, 2, 2, 0, 3, 1, 2, 0, 1, 0, 1, 1 }; + return DeBruijnBytePos[((U32)((val & -(S32)val) * 0x077CB531U)) >> 27]; +# endif + } + } + else /* Big Endian CPU */ + { + if (ZSTD_64bits()) + { +# if defined(_MSC_VER) && defined(_WIN64) && !defined(LZ4_FORCE_SW_BITCOUNT) + unsigned long r = 0; + _BitScanReverse64( &r, val ); + return (unsigned)(r>>3); +# elif defined(__GNUC__) && (GCC_VERSION >= 304) && !defined(LZ4_FORCE_SW_BITCOUNT) + return (__builtin_clzll(val) >> 3); +# else + unsigned r; + const unsigned n32 = sizeof(size_t)*4; /* calculate this way due to compiler complaining in 32-bits mode */ + if (!(val>>n32)) { r=4; } else { r=0; val>>=n32; } + if (!(val>>16)) { r+=2; val>>=8; } else { val>>=24; } + r += (!val); + return r; +# endif + } + else /* 32 bits */ + { +# if defined(_MSC_VER) && !defined(LZ4_FORCE_SW_BITCOUNT) + unsigned long r = 0; + _BitScanReverse( &r, (unsigned long)val ); + return (unsigned)(r>>3); +# elif defined(__GNUC__) && (GCC_VERSION >= 304) && !defined(LZ4_FORCE_SW_BITCOUNT) + return (__builtin_clz(val) >> 3); +# else + unsigned r; + if (!(val>>16)) { r=2; val>>=8; } else { r=0; val>>=24; } + r += (!val); + return r; +# endif + } + } +} + +static unsigned ZSTD_count(const BYTE* pIn, const BYTE* pMatch, const BYTE* pInLimit) +{ + const BYTE* const pStart = pIn; + + while ((pIn= FSE_compressBound(srcSize) > (ZSTD_blockHeaderSize+1) (checked by ZSTD_compressLiterals()) */ + (void)maxDstSize; + + ostart[ZSTD_blockHeaderSize] = *(BYTE*)src; + + /* Build header */ + ostart[0] = (BYTE)(srcSize>>16); + ostart[1] = (BYTE)(srcSize>>8); + ostart[2] = (BYTE)srcSize; + ostart[0] += (BYTE)(bt_rle<<6); + + return ZSTD_blockHeaderSize+1; +} + + +static size_t ZSTD_noCompressBlock (void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + BYTE* const ostart = (BYTE* const)dst; + + if (srcSize + ZSTD_blockHeaderSize > maxDstSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall; + memcpy(ostart + ZSTD_blockHeaderSize, src, srcSize); + + /* Build header */ + ostart[0] = (BYTE)(srcSize>>16); + ostart[1] = (BYTE)(srcSize>>8); + ostart[2] = (BYTE)srcSize; + ostart[0] += (BYTE)(bt_raw<<6); /* is a raw (uncompressed) block */ + + return ZSTD_blockHeaderSize+srcSize; +} + + +/* return : size of CStream in bits */ +static size_t ZSTD_compressLiterals_usingCTable(void* dst, size_t dstSize, + const void* src, size_t srcSize, + const void* CTable) +{ + const BYTE* const istart = (const BYTE*)src; + const BYTE* ip = istart; + const BYTE* const iend = istart + srcSize; + FSE_CStream_t bitC; + FSE_CState_t CState1, CState2; + + /* init */ + (void)dstSize; // objective : ensure it fits into dstBuffer (Todo) + FSE_initCStream(&bitC, dst); + FSE_initCState(&CState1, CTable); + CState2 = CState1; + + /* Note : at this stage, srcSize > LITERALS_NOENTROPY (checked by ZSTD_compressLiterals()) */ + // join to mod 2 + if (srcSize & 1) + { + FSE_encodeByte(&bitC, &CState1, *ip++); + FSE_flushBits(&bitC); + } + + // join to mod 4 + if ((sizeof(size_t)*8 > LitFSELog*4+7 ) && (srcSize & 2)) // test bit 2 + { + FSE_encodeByte(&bitC, &CState2, *ip++); + FSE_encodeByte(&bitC, &CState1, *ip++); + FSE_flushBits(&bitC); + } + + // 2 or 4 encoding per loop + while (ip LitFSELog*4+7 ) // this test must be static + { + FSE_encodeByte(&bitC, &CState2, *ip++); + FSE_encodeByte(&bitC, &CState1, *ip++); + } + + FSE_flushBits(&bitC); + } + + FSE_flushCState(&bitC, &CState2); + FSE_flushCState(&bitC, &CState1); + return FSE_closeCStream(&bitC); +} + + +size_t ZSTD_minGain(size_t srcSize) +{ + return (srcSize >> 6) + 1; +} + + +static size_t ZSTD_compressLiterals (void* dst, size_t dstSize, + const void* src, size_t srcSize) +{ + const BYTE* const istart = (const BYTE*) src; + const BYTE* ip = istart; + + BYTE* const ostart = (BYTE*) dst; + BYTE* op = ostart + ZSTD_blockHeaderSize; + BYTE* const oend = ostart + dstSize; + + U32 maxSymbolValue = 256; + U32 tableLog = LitFSELog; + U32 count[256]; + S16 norm[256]; + U32 CTable[ FSE_CTABLE_SIZE_U32(LitFSELog, 256) ]; + size_t errorCode; + const size_t minGain = ZSTD_minGain(srcSize); + + /* early out */ + if (dstSize < FSE_compressBound(srcSize)) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall; + + /* Scan input and build symbol stats */ + errorCode = FSE_count (count, ip, srcSize, &maxSymbolValue); + if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC; + if (errorCode == srcSize) return 1; + //if (errorCode < ((srcSize * 7) >> 10)) return 0; + //if (errorCode < (srcSize >> 7)) return 0; + if (errorCode < (srcSize >> 6)) return 0; /* heuristic : probably not compressible enough */ + + tableLog = FSE_optimalTableLog(tableLog, srcSize, maxSymbolValue); + errorCode = (int)FSE_normalizeCount (norm, tableLog, count, srcSize, maxSymbolValue); + if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC; + + /* Write table description header */ + errorCode = FSE_writeHeader (op, FSE_MAX_HEADERSIZE, norm, maxSymbolValue, tableLog); + if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC; + op += errorCode; + + /* Compress */ + errorCode = FSE_buildCTable (&CTable, norm, maxSymbolValue, tableLog); + if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC; + errorCode = ZSTD_compressLiterals_usingCTable(op, oend - op, ip, srcSize, &CTable); + if (ZSTD_isError(errorCode)) return errorCode; + op += errorCode; + + /* check compressibility */ + if ( (size_t)(op-ostart) >= srcSize-minGain) + return 0; + + /* Build header */ + { + size_t totalSize; + totalSize = op - ostart - ZSTD_blockHeaderSize; + ostart[0] = (BYTE)(totalSize>>16); + ostart[1] = (BYTE)(totalSize>>8); + ostart[2] = (BYTE)totalSize; + ostart[0] += (BYTE)(bt_compressed<<6); /* is a block, is compressed */ + } + + return op-ostart; +} + + +static size_t ZSTD_compressSequences(BYTE* dst, size_t maxDstSize, + const seqStore_t* seqStorePtr, + size_t lastLLSize, size_t srcSize) +{ + FSE_CStream_t blockStream; + U32 count[256]; + S16 norm[256]; + size_t mostFrequent; + U32 max = 255; + U32 tableLog = 11; + U32 CTable_LitLength [FSE_CTABLE_SIZE_U32(LLFSELog, MaxLL )]; + U32 CTable_OffsetBits [FSE_CTABLE_SIZE_U32(OffFSELog,MaxOff)]; + U32 CTable_MatchLength[FSE_CTABLE_SIZE_U32(MLFSELog, MaxML )]; + U32 LLtype, Offtype, MLtype; + const BYTE* const op_lit_start = seqStorePtr->litStart; + const BYTE* op_lit = seqStorePtr->lit; + const BYTE* const op_litLength_start = seqStorePtr->litLengthStart; + const BYTE* op_litLength = seqStorePtr->litLength; + const U32* op_offset = seqStorePtr->offset; + const BYTE* op_matchLength = seqStorePtr->matchLength; + const size_t nbSeq = op_litLength - op_litLength_start; + BYTE* op; + BYTE offsetBits_start[BLOCKSIZE / 4]; + BYTE* offsetBitsPtr = offsetBits_start; + const size_t minGain = ZSTD_minGain(srcSize); + const size_t maxCSize = srcSize - minGain; + const size_t minSeqSize = 1 /*lastL*/ + 2 /*dHead*/ + 2 /*dumpsIn*/ + 5 /*SeqHead*/ + 3 /*SeqIn*/ + 1 /*margin*/ + ZSTD_blockHeaderSize; + const size_t maxLSize = maxCSize > minSeqSize ? maxCSize - minSeqSize : 0; + BYTE* seqHead; + + + /* init */ + op = dst; + + /* Encode literals */ + { + size_t cSize; + size_t litSize = op_lit - op_lit_start; + if (litSize <= LITERAL_NOENTROPY) cSize = ZSTD_noCompressBlock (op, maxDstSize, op_lit_start, litSize); + else + { + cSize = ZSTD_compressLiterals(op, maxDstSize, op_lit_start, litSize); + if (cSize == 1) cSize = ZSTD_compressRle (op, maxDstSize, op_lit_start, litSize); + else if (cSize == 0) + { + if (litSize >= maxLSize) return 0; /* block not compressible enough */ + cSize = ZSTD_noCompressBlock (op, maxDstSize, op_lit_start, litSize); + } + } + if (ZSTD_isError(cSize)) return cSize; + op += cSize; + } + + /* Encode Sequences */ + + /* seqHeader */ + op += ZSTD_writeProgressive(op, lastLLSize); + seqHead = op; + + /* dumps */ + { + size_t dumpsLength = seqStorePtr->dumps - seqStorePtr->dumpsStart; + if (dumpsLength < 512) + { + op[0] = (BYTE)(dumpsLength >> 8); + op[1] = (BYTE)(dumpsLength); + op += 2; + } + else + { + op[0] = 2; + op[1] = (BYTE)(dumpsLength>>8); + op[2] = (BYTE)(dumpsLength); + op += 3; + } + memcpy(op, seqStorePtr->dumpsStart, dumpsLength); + op += dumpsLength; + } + + /* Encoding table of Literal Lengths */ + max = MaxLL; + mostFrequent = FSE_countFast(count, seqStorePtr->litLengthStart, nbSeq, &max); + if (mostFrequent == nbSeq) + { + *op++ = *(seqStorePtr->litLengthStart); + FSE_buildCTable_rle(CTable_LitLength, (BYTE)max); + LLtype = bt_rle; + } + else if ((nbSeq < 64) || (mostFrequent < (nbSeq >> (LLbits-1)))) + { + FSE_buildCTable_raw(CTable_LitLength, LLbits); + LLtype = bt_raw; + } + else + { + tableLog = FSE_optimalTableLog(LLFSELog, nbSeq, max); + FSE_normalizeCount(norm, tableLog, count, nbSeq, max); + op += FSE_writeHeader(op, maxDstSize, norm, max, tableLog); + FSE_buildCTable(CTable_LitLength, norm, max, tableLog); + LLtype = bt_compressed; + } + + /* Encoding table of Offsets */ + { + /* create OffsetBits */ + size_t i; + const U32* const op_offset_start = seqStorePtr->offsetStart; + max = MaxOff; + for (i=0; i> (Offbits-1)))) + { + FSE_buildCTable_raw(CTable_OffsetBits, Offbits); + Offtype = bt_raw; + } + else + { + tableLog = FSE_optimalTableLog(OffFSELog, nbSeq, max); + FSE_normalizeCount(norm, tableLog, count, nbSeq, max); + op += FSE_writeHeader(op, maxDstSize, norm, max, tableLog); + FSE_buildCTable(CTable_OffsetBits, norm, max, tableLog); + Offtype = bt_compressed; + } + + /* Encoding Table of MatchLengths */ + max = MaxML; + mostFrequent = FSE_countFast(count, seqStorePtr->matchLengthStart, nbSeq, &max); + if (mostFrequent == nbSeq) + { + *op++ = *seqStorePtr->matchLengthStart; + FSE_buildCTable_rle(CTable_MatchLength, (BYTE)max); + MLtype = bt_rle; + } + else if ((nbSeq < 64) || (mostFrequent < (nbSeq >> (MLbits-1)))) + { + FSE_buildCTable_raw(CTable_MatchLength, MLbits); + MLtype = bt_raw; + } + else + { + tableLog = FSE_optimalTableLog(MLFSELog, nbSeq, max); + FSE_normalizeCount(norm, tableLog, count, nbSeq, max); + op += FSE_writeHeader(op, maxDstSize, norm, max, tableLog); + FSE_buildCTable(CTable_MatchLength, norm, max, tableLog); + MLtype = bt_compressed; + } + + seqHead[0] += (BYTE)((LLtype<<6) + (Offtype<<4) + (MLtype<<2)); + + /* Encoding */ + { + FSE_CState_t stateMatchLength; + FSE_CState_t stateOffsetBits; + FSE_CState_t stateLitLength; + + FSE_initCStream(&blockStream, op); + FSE_initCState(&stateMatchLength, CTable_MatchLength); + FSE_initCState(&stateOffsetBits, CTable_OffsetBits); + FSE_initCState(&stateLitLength, CTable_LitLength); + + while (op_litLength > op_litLength_start) + { + BYTE matchLength = *(--op_matchLength); + U32 offset = *(--op_offset); + BYTE offCode = *(--offsetBitsPtr); /* 32b*/ /* 64b*/ + U32 nbBits = (offCode-1) * (!!offCode); + BYTE litLength = *(--op_litLength); /* (7)*/ /* (7)*/ + FSE_encodeByte(&blockStream, &stateMatchLength, matchLength); /* 17 */ /* 17 */ + if (ZSTD_32bits()) FSE_flushBits(&blockStream); /* 7 */ + FSE_addBits(&blockStream, offset, nbBits); /* 32 */ /* 42 */ + if (ZSTD_32bits()) FSE_flushBits(&blockStream); /* 7 */ + FSE_encodeByte(&blockStream, &stateOffsetBits, offCode); /* 16 */ /* 51 */ + FSE_encodeByte(&blockStream, &stateLitLength, litLength); /* 26 */ /* 61 */ + FSE_flushBits(&blockStream); /* 7 */ /* 7 */ + } + + FSE_flushCState(&blockStream, &stateMatchLength); + FSE_flushCState(&blockStream, &stateOffsetBits); + FSE_flushCState(&blockStream, &stateLitLength); + } + + op += FSE_closeCStream(&blockStream); + + /* check compressibility */ + if ((size_t)(op-dst) >= maxCSize) return 0; + + return op - dst; +} + + +static void ZSTD_storeSeq(seqStore_t* seqStorePtr, size_t litLength, const BYTE* literals, size_t offset, size_t matchLength) +{ + BYTE* op_lit = seqStorePtr->lit; + BYTE* const l_end = op_lit + litLength; + + /* copy Literals */ + while (op_litlit += litLength; + + /* literal Length */ + if (litLength >= MaxLL) + { + *(seqStorePtr->litLength++) = MaxLL; + if (litLength<255 + MaxLL) + *(seqStorePtr->dumps++) = (BYTE)(litLength - MaxLL); + else + { + *(seqStorePtr->dumps++) = 255; + ZSTD_writeLE32(seqStorePtr->dumps, (U32)litLength); seqStorePtr->dumps += 3; + } + } + else *(seqStorePtr->litLength++) = (BYTE)litLength; + + /* match offset */ + *(seqStorePtr->offset++) = (U32)offset; + + /* match Length */ + if (matchLength >= MaxML) + { + *(seqStorePtr->matchLength++) = MaxML; + if (matchLength < 255+MaxML) + *(seqStorePtr->dumps++) = (BYTE)(matchLength - MaxML); + else + { + *(seqStorePtr->dumps++) = 255; + ZSTD_writeLE32(seqStorePtr->dumps, (U32)matchLength); seqStorePtr->dumps+=3; + } + } + else *(seqStorePtr->matchLength++) = (BYTE)matchLength; +} + + +//static const U32 hashMask = (1<> (64-HASH_LOG)); } +//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime7bytes) >> (56-HASH_LOG)) & ((1<> (64-HASH_LOG)); } + +//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime8bytes) >> (64-HASH_LOG)); } +static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime7bytes) >> (56-HASH_LOG)) & HASH_MASK; } +//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime6bytes) >> (48-HASH_LOG)) & HASH_MASK; } +//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U64*)p * prime5bytes) >> (40-HASH_LOG)) & HASH_MASK; } +//static U32 ZSTD_hashPtr(const void* p) { return ( (*(U32*)p * KNUTH) >> (32-HASH_LOG)); } + +static void ZSTD_addPtr(U32* table, const BYTE* p, const BYTE* start) { table[ZSTD_hashPtr(p)] = (U32)(p-start); } + +static const BYTE* ZSTD_updateMatch(U32* table, const BYTE* p, const BYTE* start) +{ + U32 h = ZSTD_hashPtr(p); + const BYTE* r; + r = table[h] + start; + //table[h] = (U32)(p - start); + ZSTD_addPtr(table, p, start); + return r; +} + +static int ZSTD_checkMatch(const BYTE* match, const BYTE* ip) +{ + return ZSTD_read32(match) == ZSTD_read32(ip); +} + + +static size_t ZSTD_compressBlock(void* cctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + cctxi_t* ctx = (cctxi_t*) cctx; + U32* HashTable = (U32*)(ctx->hashTable); + seqStore_t* seqStorePtr = &(ctx->seqStore); + const BYTE* const base = ctx->base; + + const BYTE* const istart = (const BYTE*)src; + const BYTE* ip = istart + 1; + const BYTE* anchor = istart; + const BYTE* const iend = istart + srcSize; + const BYTE* const ilimit = iend - 16; + + size_t prevOffset=0, offset=0; + size_t lastLLSize; + + + /* init */ + ZSTD_resetSeqStore(seqStorePtr); + + /* Main Search Loop */ + while (ip < ilimit) + { + const BYTE* match = (BYTE*) ZSTD_updateMatch(HashTable, ip, base); + + if (!ZSTD_checkMatch(match,ip)) { ip += ((ip-anchor) >> g_searchStrength) + 1; continue; } + + /* catch up */ + while ((ip>anchor) && (match>base) && (ip[-1] == match[-1])) { ip--; match--; } + + { + size_t litLength = ip-anchor; + size_t matchLength = ZSTD_count(ip+MINMATCH, match+MINMATCH, iend); + size_t offsetCode; + if (litLength) prevOffset = offset; + offsetCode = ip-match; + if (offsetCode == prevOffset) offsetCode = 0; + prevOffset = offset; + offset = ip-match; + ZSTD_storeSeq(seqStorePtr, litLength, anchor, offsetCode, matchLength); + + /* Fill Table */ + ZSTD_addPtr(HashTable, ip+1, base); + ip += matchLength + MINMATCH; + if (ip<=iend-8) ZSTD_addPtr(HashTable, ip-2, base); + anchor = ip; + } + } + + /* Last Literals */ + lastLLSize = iend - anchor; + memcpy(seqStorePtr->lit, anchor, lastLLSize); + seqStorePtr->lit += lastLLSize; + + /* Finale compression stage */ + return ZSTD_compressSequences((BYTE*)dst, maxDstSize, + seqStorePtr, lastLLSize, srcSize); +} + + +size_t ZSTD_compressBegin(ZSTD_cctx_t ctx, void* dst, size_t maxDstSize) +{ + /* Sanity check */ + if (maxDstSize < ZSTD_frameHeaderSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall; + + /* Init */ + ZSTD_resetCCtx(ctx); + + /* Write Header */ + ZSTD_writeBE32(dst, ZSTD_magicNumber); + + return ZSTD_frameHeaderSize; +} + + +/* this should be auto-vectorized by compiler */ +static void ZSTD_scaleDownCtx(void* cctx, const U32 limit) +{ + cctxi_t* ctx = (cctxi_t*) cctx; + int i; + +#if defined(__AVX2__) /* */ + /* AVX2 version */ + __m256i* h = ctx->hashTable; + const __m256i limit8 = _mm256_set1_epi32(limit); + for (i=0; i<(HASH_TABLESIZE>>3); i++) + { + __m256i src =_mm256_loadu_si256((const __m256i*)(h+i)); + const __m256i dec = _mm256_min_epu32(src, limit8); + src = _mm256_sub_epi32(src, dec); + _mm256_storeu_si256((__m256i*)(h+i), src); + } +#else + U32* h = ctx->hashTable; + for (i=0; i limit) dec = limit; else dec = h[i]; + h[i] -= dec; + } +#endif +} + + +/* this should be auto-vectorized by compiler */ +static void ZSTD_limitCtx(void* cctx, const U32 limit) +{ + cctxi_t* ctx = (cctxi_t*) cctx; + int i; + + if (limit > g_maxLimit) + { + ZSTD_scaleDownCtx(cctx, limit); + ctx->base += limit; + ctx->current -= limit; + ctx->nextUpdate -= limit; + return; + } + +#if defined(__AVX2__) /* */ + /* AVX2 version */ + { + __m256i* h = ctx->hashTable; + const __m256i limit8 = _mm256_set1_epi32(limit); + //printf("Address h : %0X\n", (U32)h); // address test + for (i=0; i<(HASH_TABLESIZE>>3); i++) + { + __m256i src =_mm256_loadu_si256((const __m256i*)(h+i)); // Unfortunately, clang doesn't guarantee 32-bytes alignment + src = _mm256_max_epu32(src, limit8); + _mm256_storeu_si256((__m256i*)(h+i), src); + } + } +#else + { + U32* h = (U32*)(ctx->hashTable); + for (i=0; ibase==NULL) + ctx->base = (const BYTE*)src, ctx->current=0, ctx->nextUpdate = g_maxDistance; + if (src != ctx->base + ctx->current) /* not contiguous */ + { + ZSTD_resetCCtx(ctx); + ctx->base = (const BYTE*)src; + ctx->current = 0; + } + ctx->current += (U32)srcSize; + + while (srcSize) + { + size_t cSize; + size_t blockSize = BLOCKSIZE; + if (blockSize > srcSize) blockSize = srcSize; + + /* update hash table */ + if (g_maxDistance <= BLOCKSIZE) /* static test => all blocks are independent */ + { + ZSTD_resetCCtx(ctx); + ctx->base = ip; + ctx->current=0; + } + else if (ip >= ctx->base + ctx->nextUpdate) + { + ctx->nextUpdate += updateRate; + ZSTD_limitCtx(ctx, ctx->nextUpdate - g_maxDistance); + } + + /* compress */ + if (maxDstSize < ZSTD_blockHeaderSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall; + cSize = ZSTD_compressBlock(ctx, op+ZSTD_blockHeaderSize, maxDstSize-ZSTD_blockHeaderSize, ip, blockSize); + if (cSize == 0) + { + cSize = ZSTD_noCompressBlock(op, maxDstSize, ip, blockSize); /* block is not compressible */ + if (ZSTD_isError(cSize)) return cSize; + } + else + { + if (ZSTD_isError(cSize)) return cSize; + op[0] = (BYTE)(cSize>>16); + op[1] = (BYTE)(cSize>>8); + op[2] = (BYTE)cSize; + op[0] += (BYTE)(bt_compressed << 6); /* is a compressed block */ + cSize += 3; + } + op += cSize; + maxDstSize -= cSize; + ip += blockSize; + srcSize -= blockSize; + } + + return op-ostart; +} + + +size_t ZSTD_compressEnd(ZSTD_cctx_t ctx, void* dst, size_t maxDstSize) +{ + BYTE* op = (BYTE*)dst; + + /* Sanity check */ + (void)ctx; + if (maxDstSize < ZSTD_blockHeaderSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall; + + /* End of frame */ + op[0] = (BYTE)(bt_end << 6); + op[1] = 0; + op[2] = 0; + + return 3; +} + + +static size_t ZSTD_compressCCtx(void* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + BYTE* const ostart = (BYTE* const)dst; + BYTE* op = ostart; + + /* Header */ + { + size_t headerSize = ZSTD_compressBegin(ctx, dst, maxDstSize); + if(ZSTD_isError(headerSize)) return headerSize; + op += headerSize; + maxDstSize -= headerSize; + } + + /* Compression */ + { + size_t cSize = ZSTD_compressContinue(ctx, op, maxDstSize, src, srcSize); + if (ZSTD_isError(cSize)) return cSize; + op += cSize; + maxDstSize -= cSize; + } + + /* Close frame */ + { + size_t endSize = ZSTD_compressEnd(ctx, op, maxDstSize); + if(ZSTD_isError(endSize)) return endSize; + op += endSize; + } + + return (op - ostart); +} + + +size_t ZSTD_compress(void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + void* ctx; + size_t r; + + ctx = ZSTD_createCCtx(); + r = ZSTD_compressCCtx(ctx, dst, maxDstSize, src, srcSize); + ZSTD_freeCCtx(ctx); + return r; +} + + +/************************************************************** +* Decompression code +**************************************************************/ + +size_t ZSTD_getcBlockSize(const void* src, size_t srcSize, blockProperties_t* bpPtr) +{ + const BYTE* const in = (const BYTE* const)src; + BYTE headerFlags; + U32 cSize; + + if (srcSize < 3) return (size_t)-ZSTD_ERROR_wrongSrcSize; + + headerFlags = *in; + cSize = in[2] + (in[1]<<8) + ((in[0] & 7)<<16); + + bpPtr->blockType = (blockType_t)(headerFlags >> 6); + bpPtr->origSize = (bpPtr->blockType == bt_rle) ? cSize : 0; + + if (bpPtr->blockType == bt_end) return 0; + if (bpPtr->blockType == bt_rle) return 1; + return cSize; +} + + +static size_t ZSTD_copyUncompressedBlock(void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + if (srcSize > maxDstSize) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall; + memcpy(dst, src, srcSize); + return srcSize; +} + + +/* force inline : 'fast' really needs to be evaluated at compile time */ +FORCE_INLINE size_t ZSTD_decompressLiterals_usingDTable_generic( + void* const dst, size_t maxDstSize, + const void* src, size_t srcSize, + const void* DTable, U32 fast) +{ + BYTE* op = (BYTE*) dst; + BYTE* const olimit = op; + BYTE* const oend = op + maxDstSize; + FSE_DStream_t bitD; + FSE_DState_t state1, state2; + size_t errorCode; + + /* Init */ + errorCode = FSE_initDStream(&bitD, src, srcSize); + if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC; + + FSE_initDState(&state1, &bitD, DTable); + FSE_initDState(&state2, &bitD, DTable); + op = oend; + + /* 2-4 symbols per loop */ + while (!FSE_reloadDStream(&bitD) && (op>olimit+3)) + { + *--op = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD); + + if (LitFSELog*2+7 > sizeof(size_t)*8) /* This test must be static */ + FSE_reloadDStream(&bitD); + + *--op = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD); + + if (LitFSELog*4+7 < sizeof(size_t)*8) /* This test must be static */ + { + *--op = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD); + *--op = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD); + } + } + + /* tail */ + while (1) + { + if ( (FSE_reloadDStream(&bitD)>2) || (op==olimit) || (FSE_endOfDState(&state1) && FSE_endOfDStream(&bitD)) ) + break; + + *--op = fast ? FSE_decodeSymbolFast(&state1, &bitD) : FSE_decodeSymbol(&state1, &bitD); + + if ( (FSE_reloadDStream(&bitD)>2) || (op==olimit) || (FSE_endOfDState(&state2) && FSE_endOfDStream(&bitD)) ) + break; + + *--op = fast ? FSE_decodeSymbolFast(&state2, &bitD) : FSE_decodeSymbol(&state2, &bitD); + } + + /* end ? */ + if (FSE_endOfDStream(&bitD) && FSE_endOfDState(&state1) && FSE_endOfDState(&state2) ) + return oend-op; + + if (op==olimit) return (size_t)-ZSTD_ERROR_maxDstSize_tooSmall; /* dst buffer is full, but cSrc unfinished */ + + return (size_t)-ZSTD_ERROR_GENERIC; +} + +static size_t ZSTD_decompressLiterals_usingDTable( + void* const dst, size_t maxDstSize, + const void* src, size_t srcSize, + const void* DTable, U32 fast) +{ + if (fast) return ZSTD_decompressLiterals_usingDTable_generic(dst, maxDstSize, src, srcSize, DTable, 1); + return ZSTD_decompressLiterals_usingDTable_generic(dst, maxDstSize, src, srcSize, DTable, 0); +} + +static size_t ZSTD_decompressLiterals(void* ctx, void* dst, size_t maxDstSize, + const void* src, size_t srcSize) +{ + /* assumed : blockType == blockCompressed */ + const BYTE* ip = (const BYTE*)src; + short norm[256]; + void* DTable = ctx; + U32 maxSymbolValue = 255; + U32 tableLog; + U32 fastMode; + size_t errorCode; + + if (srcSize < 2) return (size_t)-ZSTD_ERROR_wrongLBlockSize; /* too small input size */ + + errorCode = FSE_readHeader (norm, &maxSymbolValue, &tableLog, ip, srcSize); + if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC; + ip += errorCode; + srcSize -= errorCode; + + errorCode = FSE_buildDTable (DTable, norm, maxSymbolValue, tableLog); + if (FSE_isError(errorCode)) return (size_t)-ZSTD_ERROR_GENERIC; + fastMode = (U32)errorCode; + + return ZSTD_decompressLiterals_usingDTable (dst, maxDstSize, ip, srcSize, DTable, fastMode); +} + + +size_t ZSTD_decodeLiteralsBlock(void* ctx, + void* dst, size_t maxDstSize, + const BYTE** litPtr, + const void* src, size_t srcSize) +{ + const BYTE* const istart = (const BYTE* const)src; + const BYTE* ip = istart; + BYTE* const ostart = (BYTE* const)dst; + BYTE* const oend = ostart + maxDstSize; + blockProperties_t litbp; + + size_t litcSize = ZSTD_getcBlockSize(src, srcSize, &litbp); + if (ZSTD_isError(litcSize)) return litcSize; + if (litcSize > srcSize - ZSTD_blockHeaderSize) return (size_t)-ZSTD_ERROR_wrongLBlockSize; + ip += ZSTD_blockHeaderSize; + + switch(litbp.blockType) + { + case bt_raw: *litPtr = ip; ip+= litcSize; break; + case bt_rle: + { + size_t rleSize = litbp.origSize; + memset(oend - rleSize, *ip, rleSize); + *litPtr = oend - rleSize; + ip++; + break; + } + case bt_compressed: + { + size_t cSize = ZSTD_decompressLiterals(ctx, dst, maxDstSize, ip, litcSize); + if (ZSTD_isError(cSize)) return cSize; + *litPtr = oend - cSize; + ip += litcSize; + break; + } + default: + return (size_t)-ZSTD_ERROR_GENERIC; + } + + return ip-istart; +} + + +size_t ZSTD_decodeSeqHeaders(size_t* lastLLPtr, const BYTE** dumpsPtr, + void* DTableLL, void* DTableML, void* DTableOffb, + const void* src, size_t srcSize) +{ + const BYTE* const istart = (const BYTE* const)src; + const BYTE* ip = istart; + const BYTE* const iend = istart + srcSize; + U32 LLtype, Offtype, MLtype; + U32 LLlog, Offlog, MLlog; + size_t dumpsLength; + + /* SeqHead */ + ip += ZSTD_readProgressive(lastLLPtr, ip); + LLtype = *ip >> 6; + Offtype = (*ip >> 4) & 3; + MLtype = (*ip >> 2) & 3; + if (*ip & 2) + { + dumpsLength = ip[2]; + dumpsLength += ip[1] << 8; + ip += 3; + } + else + { + dumpsLength = ip[1]; + dumpsLength += (ip[0] & 1) << 8; + ip += 2; + } + *dumpsPtr = ip; + ip += dumpsLength; + + /* sequences */ + { + S16 norm[MaxML+1]; /* assumption : MaxML >= MaxLL and MaxOff */ + size_t headerSize; + + /* Build DTables */ + switch(LLtype) + { + U32 max; + case bt_rle : + LLlog = 0; + FSE_buildDTable_rle(DTableLL, *ip++); break; + case bt_raw : + LLlog = LLbits; + FSE_buildDTable_raw(DTableLL, LLbits); break; + default : + max = MaxLL; + headerSize = FSE_readHeader(norm, &max, &LLlog, ip, iend-ip); + if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC; + ip += headerSize; + FSE_buildDTable(DTableLL, norm, max, LLlog); + } + + switch(Offtype) + { + U32 max; + case bt_rle : + Offlog = 0; + FSE_buildDTable_rle(DTableOffb, *ip++); break; + case bt_raw : + Offlog = Offbits; + FSE_buildDTable_raw(DTableOffb, Offbits); break; + default : + max = MaxOff; + headerSize = FSE_readHeader(norm, &max, &Offlog, ip, iend-ip); + if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC; + ip += headerSize; + FSE_buildDTable(DTableOffb, norm, max, Offlog); + } + + switch(MLtype) + { + U32 max; + case bt_rle : + MLlog = 0; + FSE_buildDTable_rle(DTableML, *ip++); break; + case bt_raw : + MLlog = MLbits; + FSE_buildDTable_raw(DTableML, MLbits); break; + default : + max = MaxML; + headerSize = FSE_readHeader(norm, &max, &MLlog, ip, iend-ip); + if (FSE_isError(headerSize)) return (size_t)-ZSTD_ERROR_GENERIC; + ip += headerSize; + FSE_buildDTable(DTableML, norm, max, MLlog); + } + } + + return ip-istart; +} + + +#define ZSTD_prefetch(p) { const BYTE pByte = *(volatile const BYTE*)p; } + +FORCE_INLINE size_t ZSTD_decompressBlock(void* ctx, void* dst, size_t maxDstSize, + const void* src, size_t srcSize) +{ + const BYTE* ip = (const BYTE*)src; + const BYTE* const iend = ip + srcSize; + BYTE* const ostart = (BYTE* const)dst; + BYTE* op = ostart; + BYTE* const oend = ostart + maxDstSize; + size_t errorCode; + size_t lastLLSize; + const BYTE* dumps; + const BYTE* litPtr; + const BYTE* litEnd; + const size_t dec32table[] = {4, 1, 2, 1, 4, 4, 4, 4}; /* added */ + const size_t dec64table[] = {8, 8, 8, 7, 8, 9,10,11}; /* substracted */ + void* DTableML = ctx; + void* DTableLL = ((U32*)ctx) + FSE_DTABLE_SIZE_U32(MLFSELog); + void* DTableOffb = ((U32*)DTableLL) + FSE_DTABLE_SIZE_U32(LLFSELog); + + /* blockType == blockCompressed, srcSize is trusted */ + + /* literal sub-block */ + errorCode = ZSTD_decodeLiteralsBlock(ctx, dst, maxDstSize, &litPtr, src, srcSize); + if (ZSTD_isError(errorCode)) return errorCode; + ip += errorCode; + + /* Build Decoding Tables */ + errorCode = ZSTD_decodeSeqHeaders(&lastLLSize, &dumps, + DTableLL, DTableML, DTableOffb, + ip, iend-ip); + if (ZSTD_isError(errorCode)) return errorCode; + /* end pos */ + if ((litPtr>=ostart) && (litPtr<=oend)) + litEnd = oend - lastLLSize; + else + litEnd = ip - lastLLSize; + ip += errorCode; + + /* decompression */ + { + FSE_DStream_t DStream; + FSE_DState_t stateLL, stateOffb, stateML; + size_t prevOffset = 0, offset = 0; + size_t qutt=0; + + FSE_initDStream(&DStream, ip, iend-ip); + FSE_initDState(&stateLL, &DStream, DTableLL); + FSE_initDState(&stateOffb, &DStream, DTableOffb); + FSE_initDState(&stateML, &DStream, DTableML); + + while (FSE_reloadDStream(&DStream)<2) + { + U32 nbBits, offsetCode; + const BYTE* match; + size_t litLength; + size_t matchLength; + size_t newOffset; + +_another_round: + + /* Literals */ + litLength = FSE_decodeSymbol(&stateLL, &DStream); + if (litLength) prevOffset = offset; + if (litLength == MaxLL) + { + BYTE add = *dumps++; + if (add < 255) litLength += add; + else + { + //litLength = (*(U32*)dumps) & 0xFFFFFF; + litLength = ZSTD_readLE32(dumps) & 0xFFFFFF; + dumps += 3; + } + } + if (((size_t)(litPtr - op) < 8) || ((size_t)(oend-(litPtr+litLength)) < 8)) + memmove(op, litPtr, litLength); /* overwrite risk */ + else + ZSTD_wildcopy(op, litPtr, litLength); + op += litLength; + litPtr += litLength; + + /* Offset */ + offsetCode = FSE_decodeSymbol(&stateOffb, &DStream); + if (ZSTD_32bits()) FSE_reloadDStream(&DStream); + nbBits = offsetCode - 1; + if (offsetCode==0) nbBits = 0; /* cmove */ + newOffset = FSE_readBits(&DStream, nbBits); + if (ZSTD_32bits()) FSE_reloadDStream(&DStream); + newOffset += (size_t)1 << nbBits; + if (offsetCode==0) newOffset = prevOffset; + match = op - newOffset; + prevOffset = offset; + offset = newOffset; + + /* MatchLength */ + matchLength = FSE_decodeSymbol(&stateML, &DStream); + if (matchLength == MaxML) + { + BYTE add = *dumps++; + if (add < 255) matchLength += add; + else + { + matchLength = ZSTD_readLE32(dumps) & 0xFFFFFF; + dumps += 3; + } + } + matchLength += MINMATCH; + + /* copy Match */ + { + BYTE* const endMatch = op + matchLength; + U64 saved[2]; + + if ((size_t)(litPtr - endMatch) < 12) + { + qutt = endMatch + 12 - litPtr; + if ((litPtr + qutt) > oend) qutt = oend-litPtr; + memcpy(saved, litPtr, qutt); + } + + if (offset < 8) + { + const size_t dec64 = dec64table[offset]; + op[0] = match[0]; + op[1] = match[1]; + op[2] = match[2]; + op[3] = match[3]; + match += dec32table[offset]; + ZSTD_copy4(op+4, match); + match -= dec64; + } else { ZSTD_copy8(op, match); } + + if (endMatch > oend-12) + { + if (op < oend-16) + { + ZSTD_wildcopy(op+8, match+8, (oend-8) - (op+8)); + match += (oend-8) - op; + op = oend-8; + } + while (op 2) return (size_t)-ZSTD_ERROR_GENERIC; /* requested too much : data is corrupted */ + if (!FSE_endOfDState(&stateLL) && !FSE_endOfDState(&stateML) && !FSE_endOfDState(&stateOffb)) goto _another_round; /* some ultra-compressible sequence remain ! */ + if (litPtr != litEnd) goto _another_round; /* literals not entirely spent */ + + /* last literal segment */ + if (op != litPtr) memmove(op, litPtr, lastLLSize); + op += lastLLSize; + } + + return op-ostart; +} + + +static size_t ZSTD_decompressDCtx(void* ctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + const BYTE* ip = (const BYTE*)src; + const BYTE* iend = ip + srcSize; + BYTE* const ostart = (BYTE* const)dst; + BYTE* op = ostart; + BYTE* const oend = ostart + maxDstSize; + size_t remainingSize = srcSize; + U32 magicNumber; + size_t errorCode=0; + blockProperties_t blockProperties; + + /* Header */ + if (srcSize < ZSTD_frameHeaderSize) return (size_t)-ZSTD_ERROR_wrongSrcSize; + magicNumber = ZSTD_readBE32(src); + if (magicNumber != ZSTD_magicNumber) return (size_t)-ZSTD_ERROR_wrongMagicNumber; + ip += ZSTD_frameHeaderSize; remainingSize -= ZSTD_frameHeaderSize; + + while (1) + { + size_t blockSize = ZSTD_getcBlockSize(ip, iend-ip, &blockProperties); + if (ZSTD_isError(blockSize)) + return blockSize; + + ip += ZSTD_blockHeaderSize; + remainingSize -= ZSTD_blockHeaderSize; + if (ip+blockSize > iend) + return (size_t)-ZSTD_ERROR_wrongSrcSize; + + switch(blockProperties.blockType) + { + case bt_compressed: + errorCode = ZSTD_decompressBlock(ctx, op, oend-op, ip, blockSize); + break; + case bt_raw : + errorCode = ZSTD_copyUncompressedBlock(op, oend-op, ip, blockSize); + break; + case bt_rle : + return (size_t)-ZSTD_ERROR_GENERIC; /* not yet handled */ + break; + case bt_end : + /* end of frame */ + if (remainingSize) return (size_t)-ZSTD_ERROR_wrongSrcSize; + break; + default: + return (size_t)-ZSTD_ERROR_GENERIC; + } + if (blockSize == 0) break; /* bt_end */ + + if (ZSTD_isError(errorCode)) return errorCode; + op += errorCode; + ip += blockSize; + remainingSize -= blockSize; + } + + return op-ostart; +} + + +size_t ZSTD_decompress(void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + U32 ctx[FSE_DTABLE_SIZE_U32(LLFSELog) + FSE_DTABLE_SIZE_U32(OffFSELog) + FSE_DTABLE_SIZE_U32(MLFSELog)]; + return ZSTD_decompressDCtx(ctx, dst, maxDstSize, src, srcSize); +} + + +/******************************* +* Streaming Decompression API +*******************************/ + +typedef struct +{ + U32 ctx[FSE_DTABLE_SIZE_U32(LLFSELog) + FSE_DTABLE_SIZE_U32(OffFSELog) + FSE_DTABLE_SIZE_U32(MLFSELog)]; + size_t expected; + blockType_t bType; + U32 phase; +} dctx_t; + + +ZSTD_dctx_t ZSTD_createDCtx(void) +{ + dctx_t* dctx = (dctx_t*)malloc(sizeof(dctx_t)); + dctx->expected = ZSTD_frameHeaderSize; + dctx->phase = 0; + return (ZSTD_dctx_t)dctx; +} + +size_t ZSTD_freeDCtx(ZSTD_dctx_t dctx) +{ + free(dctx); + return 0; +} + + +size_t ZSTD_nextSrcSizeToDecompress(ZSTD_dctx_t dctx) +{ + return ((dctx_t*)dctx)->expected; +} + +size_t ZSTD_decompressContinue(ZSTD_dctx_t dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize) +{ + dctx_t* ctx = (dctx_t*)dctx; + + /* Sanity check */ + if (srcSize != ctx->expected) return (size_t)-ZSTD_ERROR_wrongSrcSize; + + /* Decompress : frame header */ + if (ctx->phase == 0) + { + /* Check frame magic header */ + U32 magicNumber = ZSTD_readBE32(src); + if (magicNumber != ZSTD_magicNumber) return (size_t)-ZSTD_ERROR_wrongMagicNumber; + ctx->phase = 1; + ctx->expected = ZSTD_blockHeaderSize; + return 0; + } + + /* Decompress : block header */ + if (ctx->phase == 1) + { + blockProperties_t bp; + size_t blockSize = ZSTD_getcBlockSize(src, ZSTD_blockHeaderSize, &bp); + if (ZSTD_isError(blockSize)) return blockSize; + if (bp.blockType == bt_end) + { + ctx->expected = 0; + ctx->phase = 0; + } + else + { + ctx->expected = blockSize; + ctx->bType = bp.blockType; + ctx->phase = 2; + } + + return 0; + } + + /* Decompress : block content */ + { + size_t rSize; + switch(ctx->bType) + { + case bt_compressed: + rSize = ZSTD_decompressBlock(ctx, dst, maxDstSize, src, srcSize); + break; + case bt_raw : + rSize = ZSTD_copyUncompressedBlock(dst, maxDstSize, src, srcSize); + break; + case bt_rle : + return (size_t)-ZSTD_ERROR_GENERIC; /* not yet handled */ + break; + case bt_end : /* should never happen (filtered at phase 1) */ + rSize = 0; + break; + default: + return (size_t)-ZSTD_ERROR_GENERIC; + } + ctx->phase = 1; + ctx->expected = ZSTD_blockHeaderSize; + return rSize; + } + +} + + diff --git a/libs/libzstd/include/zstd/zstd.h b/libs/libzstd/include/zstd/zstd.h new file mode 100644 index 00000000000..cda16b88ad2 --- /dev/null +++ b/libs/libzstd/include/zstd/zstd.h @@ -0,0 +1,93 @@ +/* + zstd - standard compression library + Header File + Copyright (C) 2014-2015, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - zstd source repository : https://github.com/Cyan4973/zstd + - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c +*/ +#pragma once + +#if defined (__cplusplus) +extern "C" { +#endif + +/************************************** +* Includes +**************************************/ +#include /* size_t */ + + +/************************************** +* Version +**************************************/ +#define ZSTD_VERSION_MAJOR 0 /* for breaking interface changes */ +#define ZSTD_VERSION_MINOR 0 /* for new (non-breaking) interface capabilities */ +#define ZSTD_VERSION_RELEASE 2 /* for tweaks, bug-fixes, or development */ +#define ZSTD_VERSION_NUMBER (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE) +unsigned ZSTD_versionNumber (void); + + +/************************************** +* Simple one-step functions +**************************************/ +size_t ZSTD_compress( void* dst, size_t maxDstSize, + const void* src, size_t srcSize); + +size_t ZSTD_decompress( void* dst, size_t maxOriginalSize, + const void* src, size_t compressedSize); + +/* +ZSTD_compress() : + Compresses 'srcSize' bytes from buffer 'src' into buffer 'dst', of maximum size 'dstSize'. + Destination buffer should be sized to handle worst cases situations (input data not compressible). + Worst case size evaluation is provided by function ZSTD_compressBound(). + return : the number of bytes written into buffer 'dst' + or an error code if it fails (which can be tested using ZSTD_isError()) + +ZSTD_decompress() : + compressedSize : is obviously the source size + maxOriginalSize : is the size of the 'dst' buffer, which must be already allocated. + It must be equal or larger than originalSize, otherwise decompression will fail. + return : the number of bytes decompressed into destination buffer (originalSize) + or an errorCode if it fails (which can be tested using ZSTD_isError()) +*/ + + +/************************************** +* Tool functions +**************************************/ +size_t ZSTD_compressBound(size_t srcSize); /* maximum compressed size */ + +/* Error Management */ +unsigned ZSTD_isError(size_t code); /* tells if a return value is an error code */ +const char* ZSTD_getErrorName(size_t code); /* provides error code string (useful for debugging) */ + + +#if defined (__cplusplus) +} +#endif diff --git a/libs/libzstd/include/zstd/zstd_static.h b/libs/libzstd/include/zstd/zstd_static.h new file mode 100644 index 00000000000..b63fbba06b0 --- /dev/null +++ b/libs/libzstd/include/zstd/zstd_static.h @@ -0,0 +1,87 @@ +/* + zstd - standard compression library + Header File for static linking only + Copyright (C) 2014-2015, Yann Collet. + + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - zstd source repository : https://github.com/Cyan4973/zstd + - ztsd public forum : https://groups.google.com/forum/#!forum/lz4c +*/ +#pragma once + +#if defined (__cplusplus) +extern "C" { +#endif + +/************************************** +* Includes +**************************************/ +#include "zstd.h" + + +/************************************** +* Streaming functions +**************************************/ +typedef void* ZSTD_cctx_t; +ZSTD_cctx_t ZSTD_createCCtx(void); +size_t ZSTD_freeCCtx(ZSTD_cctx_t cctx); + +size_t ZSTD_compressBegin(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize); +size_t ZSTD_compressContinue(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize); +size_t ZSTD_compressEnd(ZSTD_cctx_t cctx, void* dst, size_t maxDstSize); + +typedef void* ZSTD_dctx_t; +ZSTD_dctx_t ZSTD_createDCtx(void); +size_t ZSTD_freeDCtx(ZSTD_dctx_t dctx); + +size_t ZSTD_nextSrcSizeToDecompress(ZSTD_dctx_t dctx); +size_t ZSTD_decompressContinue(ZSTD_dctx_t dctx, void* dst, size_t maxDstSize, const void* src, size_t srcSize); +/* + Use above functions alternatively. + ZSTD_nextSrcSizeToDecompress() tells how much bytes to provide as input to ZSTD_decompressContinue(). + This value is expected to be provided, precisely, as 'srcSize'. + Otherwise, compression will fail (result is an error code, which can be tested using ZSTD_isError() ) + ZSTD_decompressContinue() result is the number of bytes regenerated within 'dst'. + It can be zero, which is not an error; it just means ZSTD_decompressContinue() has decoded some header. +*/ + +/************************************** +* Error management +**************************************/ +#define ZSTD_LIST_ERRORS(ITEM) \ + ITEM(ZSTD_OK_NoError) ITEM(ZSTD_ERROR_GENERIC) \ + ITEM(ZSTD_ERROR_wrongMagicNumber) \ + ITEM(ZSTD_ERROR_wrongSrcSize) ITEM(ZSTD_ERROR_maxDstSize_tooSmall) \ + ITEM(ZSTD_ERROR_wrongLBlockSize) \ + ITEM(ZSTD_ERROR_maxCode) + +#define ZSTD_GENERATE_ENUM(ENUM) ENUM, +typedef enum { ZSTD_LIST_ERRORS(ZSTD_GENERATE_ENUM) } ZSTD_errorCodes; /* exposed list of errors; static linking only */ + + +#if defined (__cplusplus) +} +#endif diff --git a/utils/compressor/main.cpp b/utils/compressor/main.cpp index 000606bbb34..e4e92f327a7 100644 --- a/utils/compressor/main.cpp +++ b/utils/compressor/main.cpp @@ -45,6 +45,7 @@ int main(int argc, char ** argv) ("block-size,b", boost::program_options::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("qlz", "use QuickLZ (level 1) instead of LZ4") + ("zstd", "use ZSTD instead of LZ4") ("stat", "print block statistics of compressed data") ; @@ -61,17 +62,20 @@ int main(int argc, char ** argv) try { bool decompress = options.count("d"); - bool use_qlz = options.count("qlz");; - bool use_lz4hc = options.count("hc");; + bool use_qlz = options.count("qlz"); + bool use_lz4hc = options.count("hc"); + bool use_zstd = options.count("zstd"); bool stat_mode = options.count("stat"); unsigned block_size = options["block-size"].as(); - DB::CompressionMethod::Enum method = DB::CompressionMethod::LZ4; + DB::CompressionMethod method = DB::CompressionMethod::LZ4; if (use_qlz) method = DB::CompressionMethod::QuickLZ; else if (use_lz4hc) method = DB::CompressionMethod::LZ4HC; + else if (use_zstd) + method = DB::CompressionMethod::ZSTD; DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO); DB::WriteBufferFromFileDescriptor wb(STDOUT_FILENO);