aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorchertus <azuikov@ydb.tech>2022-08-25 16:08:44 +0300
committerchertus <azuikov@ydb.tech>2022-08-25 16:08:44 +0300
commitea542b16aeaff26273efaba14569b1d35a24a0ee (patch)
treeda30600f503c4f4d08f9b2194f1b4f1438a30fd0
parenta9cf4253d01d3c7ef62eeb9e798db23942ef1bcd (diff)
downloadydb-ea542b16aeaff26273efaba14569b1d35a24a0ee.tar.gz
ClickHouse aggregates functions library over Apache Arrow primitives
-rw-r--r--CMakeLists.darwin.txt5
-rw-r--r--CMakeLists.linux.txt5
-rw-r--r--ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionAvg.h198
-rw-r--r--ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionCount.h93
-rw-r--r--ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionMinMaxAny.h682
-rw-r--r--ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionSum.h302
-rw-r--r--ydb/library/arrow_clickhouse/AggregateFunctions/IAggregateFunction.h495
-rw-r--r--ydb/library/arrow_clickhouse/AggregationCommon.h337
-rw-r--r--ydb/library/arrow_clickhouse/Aggregator.cpp1554
-rw-r--r--ydb/library/arrow_clickhouse/Aggregator.h923
-rw-r--r--ydb/library/arrow_clickhouse/CMakeLists.txt29
-rw-r--r--ydb/library/arrow_clickhouse/Columns/CMakeLists.txt23
-rw-r--r--ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.cpp31
-rw-r--r--ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.h150
-rw-r--r--ydb/library/arrow_clickhouse/Columns/ColumnsCommon.cpp530
-rw-r--r--ydb/library/arrow_clickhouse/Columns/ColumnsCommon.h120
-rw-r--r--ydb/library/arrow_clickhouse/Columns/ColumnsHashing.h293
-rw-r--r--ydb/library/arrow_clickhouse/Columns/ColumnsHashingImpl.h375
-rw-r--r--ydb/library/arrow_clickhouse/Common/Allocator.cpp35
-rw-r--r--ydb/library/arrow_clickhouse/Common/Allocator.h344
-rw-r--r--ydb/library/arrow_clickhouse/Common/Allocator_fwd.h19
-rw-r--r--ydb/library/arrow_clickhouse/Common/Arena.h311
-rw-r--r--ydb/library/arrow_clickhouse/Common/CMakeLists.txt23
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/FixedHashMap.h186
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/FixedHashTable.h497
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/Hash.h417
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/HashMap.h314
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/HashSet.h124
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/HashTable.h1311
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/HashTableAllocator.h22
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/HashTableKeyHolder.h127
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/StringHashMap.h198
-rw-r--r--ydb/library/arrow_clickhouse/Common/HashTable/StringHashTable.h444
-rw-r--r--ydb/library/arrow_clickhouse/Common/PODArray.cpp23
-rw-r--r--ydb/library/arrow_clickhouse/Common/PODArray.h803
-rw-r--r--ydb/library/arrow_clickhouse/Common/PODArray_fwd.h40
-rw-r--r--ydb/library/arrow_clickhouse/Common/SipHash.h219
-rw-r--r--ydb/library/arrow_clickhouse/Common/memcpySmall.h78
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.cpp127
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.h49
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/CMakeLists.txt24
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.cpp90
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.h118
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/IBlockStream_fwd.h21
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.cpp43
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.h42
-rw-r--r--ydb/library/arrow_clickhouse/DataStreams/OneBlockInputStream.h49
-rw-r--r--ydb/library/arrow_clickhouse/README.md16
-rw-r--r--ydb/library/arrow_clickhouse/arrow_clickhouse_types.h146
-rw-r--r--ydb/library/arrow_clickhouse/ut/CMakeLists.darwin.txt41
-rw-r--r--ydb/library/arrow_clickhouse/ut/CMakeLists.linux.txt47
-rw-r--r--ydb/library/arrow_clickhouse/ut/CMakeLists.txt13
-rw-r--r--ydb/library/arrow_clickhouse/ut_aggregator.cpp365
53 files changed, 12871 insertions, 0 deletions
diff --git a/CMakeLists.darwin.txt b/CMakeLists.darwin.txt
index c55bc4077e2..a2f25521c2f 100644
--- a/CMakeLists.darwin.txt
+++ b/CMakeLists.darwin.txt
@@ -1043,6 +1043,10 @@ add_subdirectory(ydb/core/filestore)
add_subdirectory(ydb/core/grpc_caching)
add_subdirectory(ydb/core/pgproxy)
add_subdirectory(ydb/core/yql_testlib)
+add_subdirectory(ydb/library/arrow_clickhouse)
+add_subdirectory(ydb/library/arrow_clickhouse/Common)
+add_subdirectory(ydb/library/arrow_clickhouse/Columns)
+add_subdirectory(ydb/library/arrow_clickhouse/DataStreams)
add_subdirectory(ydb/core/actorlib_impl/ut)
add_subdirectory(library/cpp/testing/unittest_main)
add_subdirectory(library/cpp/terminate_handler)
@@ -1109,6 +1113,7 @@ add_subdirectory(ydb/core/wrappers/ut)
add_subdirectory(ydb/core/ydb_convert/ut)
add_subdirectory(ydb/core/ymq/ut)
add_subdirectory(ydb/library/aclib/ut)
+add_subdirectory(ydb/library/arrow_clickhouse/ut)
add_subdirectory(ydb/library/backup/ut)
add_subdirectory(ydb/library/binary_json/ut)
add_subdirectory(ydb/library/dynumber/ut)
diff --git a/CMakeLists.linux.txt b/CMakeLists.linux.txt
index 325b788019c..a323ecbd724 100644
--- a/CMakeLists.linux.txt
+++ b/CMakeLists.linux.txt
@@ -1047,6 +1047,10 @@ add_subdirectory(ydb/core/filestore)
add_subdirectory(ydb/core/grpc_caching)
add_subdirectory(ydb/core/pgproxy)
add_subdirectory(ydb/core/yql_testlib)
+add_subdirectory(ydb/library/arrow_clickhouse)
+add_subdirectory(ydb/library/arrow_clickhouse/Common)
+add_subdirectory(ydb/library/arrow_clickhouse/Columns)
+add_subdirectory(ydb/library/arrow_clickhouse/DataStreams)
add_subdirectory(ydb/core/actorlib_impl/ut)
add_subdirectory(library/cpp/testing/unittest_main)
add_subdirectory(library/cpp/terminate_handler)
@@ -1113,6 +1117,7 @@ add_subdirectory(ydb/core/wrappers/ut)
add_subdirectory(ydb/core/ydb_convert/ut)
add_subdirectory(ydb/core/ymq/ut)
add_subdirectory(ydb/library/aclib/ut)
+add_subdirectory(ydb/library/arrow_clickhouse/ut)
add_subdirectory(ydb/library/backup/ut)
add_subdirectory(ydb/library/binary_json/ut)
add_subdirectory(ydb/library/dynumber/ut)
diff --git a/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionAvg.h b/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionAvg.h
new file mode 100644
index 00000000000..3e8bce5fdbb
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionAvg.h
@@ -0,0 +1,198 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <type_traits>
+
+#include <AggregateFunctions/IAggregateFunction.h>
+#include <AggregateFunctions/AggregateFunctionSum.h>
+
+namespace CH
+{
+
+/**
+ * Helper class to encapsulate values conversion for avg and avgWeighted.
+ */
+template <typename Numerator, typename Denominator>
+struct AvgFraction
+{
+ Numerator numerator{0};
+ Denominator denominator{0};
+
+ double divide() const
+ {
+ return static_cast<double>(numerator) / denominator;
+ }
+};
+
+
+/**
+ * @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g.
+ * class Self : Agg<char, bool, bool, Self>.
+ */
+template <typename TNumerator, typename TDenominator, typename Derived>
+class AggregateFunctionAvgBase : public
+ IAggregateFunctionDataHelper<AvgFraction<TNumerator, TDenominator>, Derived>
+{
+public:
+ using Base = IAggregateFunctionDataHelper<AvgFraction<TNumerator, TDenominator>, Derived>;
+ using Numerator = TNumerator;
+ using Denominator = TDenominator;
+ using Fraction = AvgFraction<Numerator, Denominator>;
+
+ explicit AggregateFunctionAvgBase(const DataTypes & argument_types_,
+ UInt32 num_scale_ = 0, UInt32 denom_scale_ = 0)
+ : Base(argument_types_, {}), num_scale(num_scale_), denom_scale(denom_scale_) {}
+
+ DataTypePtr getReturnType() const override
+ {
+ return std::make_shared<arrow::DoubleType>();
+ }
+
+ bool allocatesMemoryInArena() const override { return false; }
+
+ void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
+ {
+ this->data(place).numerator += this->data(rhs).numerator;
+ this->data(place).denominator += this->data(rhs).denominator;
+ }
+#if 0
+ void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
+ {
+ writeBinary(this->data(place).numerator, buf);
+
+ if constexpr (std::is_unsigned_v<Denominator>)
+ writeVarUInt(this->data(place).denominator, buf);
+ else /// Floating point denominator type can be used
+ writeBinary(this->data(place).denominator, buf);
+ }
+
+ void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
+ {
+ readBinary(this->data(place).numerator, buf);
+
+ if constexpr (std::is_unsigned_v<Denominator>)
+ readVarUInt(this->data(place).denominator, buf);
+ else /// Floating point denominator type can be used
+ readBinary(this->data(place).denominator, buf);
+ }
+#endif
+ void insertResultInto(AggregateDataPtr __restrict place, MutableColumn & to, Arena *) const override
+ {
+ assert_cast<MutableColumnFloat64 &>(to).Append(this->data(place).divide()).ok();
+ }
+
+private:
+ UInt32 num_scale;
+ UInt32 denom_scale;
+};
+
+template <typename T>
+using AvgFieldType = std::conditional_t<std::is_floating_point_v<T>, T, UInt64>;
+
+template <typename T>
+class AggregateFunctionAvg : public AggregateFunctionAvgBase<AvgFieldType<T>, UInt64, AggregateFunctionAvg<T>>
+{
+public:
+ using Base = AggregateFunctionAvgBase<AvgFieldType<T>, UInt64, AggregateFunctionAvg<T>>;
+ using Base::Base;
+
+ using Numerator = typename Base::Numerator;
+ using Denominator = typename Base::Denominator;
+ using Fraction = typename Base::Fraction;
+
+ using ColumnType = arrow::NumericArray<T>;
+ using MutableColumnType = arrow::NumericBuilder<T>;
+
+ void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const final
+ {
+ increment(place, static_cast<const ColumnType &>(*columns[0]).Value(row_num));
+ ++this->data(place).denominator;
+ }
+
+ void addBatchSinglePlace(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** columns,
+ Arena *,
+ ssize_t if_argument_pos) const final
+ {
+ AggregateFunctionSumData<Numerator> sum_data;
+ const auto & column = assert_cast<const ColumnType &>(*columns[0]);
+ if (if_argument_pos >= 0)
+ {
+ const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).raw_values();
+ sum_data.addManyConditional(column.raw_values(), flags, row_begin, row_end);
+ this->data(place).denominator += countBytesInFilter(flags, row_begin, row_end);
+ }
+ else
+ {
+ sum_data.addMany(column.raw_values(), row_begin, row_end);
+ this->data(place).denominator += (row_end - row_begin);
+ }
+ increment(place, sum_data.sum);
+ }
+
+private:
+ void increment(AggregateDataPtr __restrict place, Numerator inc) const
+ {
+ this->data(place).numerator += inc;
+ }
+};
+
+class WrappedAvg final : public ArrowAggregateFunctionWrapper
+{
+public:
+ WrappedAvg(std::string name)
+ : ArrowAggregateFunctionWrapper(std::move(name))
+ {}
+
+ AggregateFunctionPtr getHouseFunction(const DataTypes & argument_types) override
+ {
+ return createWithSameType<AggregateFunctionAvg>(argument_types);
+ }
+
+ template <template <typename> typename AggFunc>
+ std::shared_ptr<IAggregateFunction> createWithSameType(const DataTypes & argument_types)
+ {
+ if (argument_types.size() != 1)
+ return {};
+
+ const DataTypePtr & type = argument_types[0];
+
+ switch (type->id()) {
+ case arrow::Type::INT8:
+ return std::make_shared<AggFunc<arrow::Int8Type>>(argument_types);
+ case arrow::Type::INT16:
+ return std::make_shared<AggFunc<arrow::Int16Type>>(argument_types);
+ case arrow::Type::INT32:
+ return std::make_shared<AggFunc<arrow::Int32Type>>(argument_types);
+ case arrow::Type::INT64:
+ return std::make_shared<AggFunc<arrow::Int64Type>>(argument_types);
+ case arrow::Type::UINT8:
+ return std::make_shared<AggFunc<arrow::UInt8Type>>(argument_types);
+ case arrow::Type::UINT16:
+ return std::make_shared<AggFunc<arrow::UInt16Type>>(argument_types);
+ case arrow::Type::UINT32:
+ return std::make_shared<AggFunc<arrow::UInt32Type>>(argument_types);
+ case arrow::Type::UINT64:
+ return std::make_shared<AggFunc<arrow::UInt64Type>>(argument_types);
+ case arrow::Type::FLOAT:
+ return std::make_shared<AggFunc<arrow::FloatType>>(argument_types);
+ case arrow::Type::DOUBLE:
+ return std::make_shared<AggFunc<arrow::DoubleType>>(argument_types);
+ case arrow::Type::DURATION:
+ return std::make_shared<AggFunc<arrow::DurationType>>(argument_types);
+ default:
+ break;
+ }
+
+ return {};
+ }
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionCount.h b/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionCount.h
new file mode 100644
index 00000000000..6044df828e6
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionCount.h
@@ -0,0 +1,93 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <AggregateFunctions/IAggregateFunction.h>
+#include <Columns/ColumnsCommon.h>
+
+#include <array>
+
+namespace CH
+{
+
+
+struct AggregateFunctionCountData
+{
+ UInt64 count = 0;
+};
+
+
+/// Simply count number of calls.
+class AggregateFunctionCount final : public IAggregateFunctionDataHelper<AggregateFunctionCountData, AggregateFunctionCount>
+{
+public:
+ AggregateFunctionCount(const DataTypes & argument_types_)
+ : IAggregateFunctionDataHelper(argument_types_, {})
+ {}
+
+ DataTypePtr getReturnType() const override
+ {
+ return std::make_shared<DataTypeUInt64>();
+ }
+
+ bool allocatesMemoryInArena() const override { return false; }
+
+ void add(AggregateDataPtr __restrict place, const IColumn **, size_t, Arena *) const override
+ {
+ ++data(place).count;
+ }
+
+ void addBatchSinglePlace(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** columns,
+ Arena *,
+ ssize_t if_argument_pos) const override
+ {
+ if (if_argument_pos >= 0)
+ {
+ const auto & filter_column = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]);
+ const auto & flags = filter_column.raw_values();
+ data(place).count += countBytesInFilter(flags, row_begin, row_end);
+ }
+ else
+ {
+ data(place).count += row_end - row_begin;
+ }
+ }
+
+ void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
+ {
+ data(place).count += data(rhs).count;
+ }
+
+ void insertResultInto(AggregateDataPtr __restrict place, MutableColumn & to, Arena *) const override
+ {
+ assert_cast<MutableColumnUInt64 &>(to).Append(data(place).count).ok();
+ }
+
+ /// Reset the state to specified value. This function is not the part of common interface.
+ void set(AggregateDataPtr __restrict place, UInt64 new_count) const
+ {
+ data(place).count = new_count;
+ }
+};
+
+class WrappedCount final : public ArrowAggregateFunctionWrapper
+{
+public:
+ WrappedCount(std::string name)
+ : ArrowAggregateFunctionWrapper(std::move(name))
+ {}
+
+ AggregateFunctionPtr getHouseFunction(const DataTypes & argument_types) override
+ {
+ return std::make_shared<AggregateFunctionCount>(argument_types);
+ }
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionMinMaxAny.h b/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionMinMaxAny.h
new file mode 100644
index 00000000000..5fbfb53170d
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionMinMaxAny.h
@@ -0,0 +1,682 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <AggregateFunctions/IAggregateFunction.h>
+
+
+namespace CH
+{
+
+/// For numeric values.
+template <typename ArrowType>
+struct SingleValueDataFixed
+{
+private:
+ using Self = SingleValueDataFixed;
+ using ColumnType = arrow::NumericArray<ArrowType>;
+ using MutableColumnType = arrow::NumericBuilder<ArrowType>;
+
+ bool has_value = false; /// We need to remember if at least one value has been passed. This is necessary for AggregateFunctionIf.
+ typename arrow::TypeTraits<ArrowType>::CType value;
+
+public:
+ static constexpr bool is_any = false;
+
+ bool has() const
+ {
+ return has_value;
+ }
+
+ void insertResultInto(MutableColumn & to) const
+ {
+ if (has())
+ assert_cast<MutableColumnType &>(to).Append(value).ok();
+ else
+ assert_cast<MutableColumnType &>(to).AppendEmptyValue().ok();
+ }
+
+ void change(const IColumn & column, size_t row_num, Arena *)
+ {
+ has_value = true;
+ value = assert_cast<const ColumnType &>(column).Value(row_num);
+ }
+
+ /// Assuming to.has()
+ void change(const Self & to, Arena *)
+ {
+ has_value = true;
+ value = to.value;
+ }
+
+ bool changeFirstTime(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has())
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeFirstTime(const Self & to, Arena * arena)
+ {
+ if (!has() && to.has())
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeEveryTime(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+
+ bool changeEveryTime(const Self & to, Arena * arena)
+ {
+ if (to.has())
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfLess(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has() || assert_cast<const ColumnType &>(column).Value(row_num) < value)
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfLess(const Self & to, Arena * arena)
+ {
+ if (to.has() && (!has() || to.value < value))
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has() || assert_cast<const ColumnType &>(column).Value(row_num) > value)
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfGreater(const Self & to, Arena * arena)
+ {
+ if (to.has() && (!has() || to.value > value))
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool isEqualTo(const Self & to) const
+ {
+ return has() && to.value == value;
+ }
+
+ bool isEqualTo(const IColumn & column, size_t row_num) const
+ {
+ return has() && assert_cast<const ColumnType &>(column).Value(row_num) == value;
+ }
+
+ static bool allocatesMemoryInArena()
+ {
+ return false;
+ }
+};
+
+
+/** For strings. Short strings are stored in the object itself, and long strings are allocated separately.
+ * NOTE It could also be suitable for arrays of numbers.
+ */
+template <bool is_utf8_string>
+struct SingleValueDataString
+{
+private:
+ using Self = SingleValueDataString<is_utf8_string>;
+ using ColumnType = std::conditional_t<is_utf8_string, ColumnString, ColumnBinary>;
+ using MutableColumnType = std::conditional_t<is_utf8_string, MutableColumnString, MutableColumnBinary>;
+
+ Int32 size = -1; /// -1 indicates that there is no value.
+ Int32 capacity = 0; /// power of two or zero
+ char * large_data;
+
+public:
+ static constexpr Int32 AUTOMATIC_STORAGE_SIZE = 64;
+ static constexpr Int32 MAX_SMALL_STRING_SIZE = AUTOMATIC_STORAGE_SIZE - sizeof(size) - sizeof(capacity) - sizeof(large_data);
+
+private:
+ char small_data[MAX_SMALL_STRING_SIZE]; /// Including the terminating zero.
+
+public:
+ static constexpr bool is_any = false;
+
+ bool has() const
+ {
+ return size >= 0;
+ }
+
+ const char * getData() const
+ {
+ return size <= MAX_SMALL_STRING_SIZE ? small_data : large_data;
+ }
+
+ arrow::util::string_view getStringView() const
+ {
+ if (!has())
+ return {};
+ return arrow::util::string_view(getData(), size);
+ }
+
+ void insertResultInto(MutableColumn & to) const
+ {
+ if (has())
+ assert_cast<MutableColumnType &>(to).Append(getData(), size).ok();
+ else
+ assert_cast<MutableColumnType &>(to).AppendEmptyValue().ok();
+ }
+
+ /// Assuming to.has()
+ void changeImpl(arrow::util::string_view value, Arena * arena)
+ {
+ Int32 value_size = value.size();
+
+ if (value_size <= MAX_SMALL_STRING_SIZE)
+ {
+ /// Don't free large_data here.
+ size = value_size;
+
+ if (size > 0)
+ memcpy(small_data, value.data(), size);
+ }
+ else
+ {
+ if (capacity < value_size)
+ {
+ /// Don't free large_data here.
+ capacity = roundUpToPowerOfTwoOrZero(value_size);
+ large_data = arena->alloc(capacity);
+ }
+
+ size = value_size;
+ memcpy(large_data, value.data(), size);
+ }
+ }
+
+ void change(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ changeImpl(assert_cast<const ColumnType &>(column).Value(row_num), arena);
+ }
+
+ void change(const Self & to, Arena * arena)
+ {
+ changeImpl(to.getStringView(), arena);
+ }
+
+ bool changeFirstTime(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has())
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeFirstTime(const Self & to, Arena * arena)
+ {
+ if (!has() && to.has())
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeEveryTime(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+
+ bool changeEveryTime(const Self & to, Arena * arena)
+ {
+ if (to.has())
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfLess(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has() || assert_cast<const ColumnType &>(column).Value(row_num) < getStringView())
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfLess(const Self & to, Arena * arena)
+ {
+ if (to.has() && (!has() || to.getStringView() < getStringView()))
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has() || assert_cast<const ColumnType &>(column).Value(row_num) > getStringView())
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfGreater(const Self & to, Arena * arena)
+ {
+ if (to.has() && (!has() || to.getStringView() > getStringView()))
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool isEqualTo(const Self & to) const
+ {
+ return has() && to.getStringView() == getStringView();
+ }
+
+ bool isEqualTo(const IColumn & column, size_t row_num) const
+ {
+ return has() && assert_cast<const ColumnType &>(column).Value(row_num) == getStringView();
+ }
+
+ static bool allocatesMemoryInArena()
+ {
+ return true;
+ }
+};
+
+static_assert(sizeof(SingleValueDataString<false>) == SingleValueDataString<false>::AUTOMATIC_STORAGE_SIZE,
+ "Incorrect size of SingleValueDataString struct");
+static_assert(sizeof(SingleValueDataString<true>) == SingleValueDataString<true>::AUTOMATIC_STORAGE_SIZE,
+ "Incorrect size of SingleValueDataString struct");
+
+
+#if 0
+/// For any other value types.
+struct SingleValueDataGeneric
+{
+private:
+ using Self = SingleValueDataGeneric;
+ static constexpr bool is_any = false;
+
+ Field value;
+
+public:
+ bool has() const
+ {
+ return !value.isNull();
+ }
+
+ void insertResultInto(IColumn & to) const
+ {
+ if (has())
+ to.insert(value);
+ else
+ to.insertDefault();
+ }
+
+ void change(const IColumn & column, size_t row_num, Arena *)
+ {
+ column.get(row_num, value);
+ }
+
+ void change(const Self & to, Arena *)
+ {
+ value = to.value;
+ }
+
+ bool changeFirstTime(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has())
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeFirstTime(const Self & to, Arena * arena)
+ {
+ if (!has() && to.has())
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeEveryTime(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+
+ bool changeEveryTime(const Self & to, Arena * arena)
+ {
+ if (to.has())
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfLess(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has())
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ {
+ Field new_value;
+ column.get(row_num, new_value);
+ if (new_value < value)
+ {
+ value = new_value;
+ return true;
+ }
+ else
+ return false;
+ }
+ }
+
+ bool changeIfLess(const Self & to, Arena * arena)
+ {
+ if (to.has() && (!has() || to.value < value))
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena)
+ {
+ if (!has())
+ {
+ change(column, row_num, arena);
+ return true;
+ }
+ else
+ {
+ Field new_value;
+ column.get(row_num, new_value);
+ if (new_value > value)
+ {
+ value = new_value;
+ return true;
+ }
+ else
+ return false;
+ }
+ }
+
+ bool changeIfGreater(const Self & to, Arena * arena)
+ {
+ if (to.has() && (!has() || to.value > value))
+ {
+ change(to, arena);
+ return true;
+ }
+ else
+ return false;
+ }
+
+ bool isEqualTo(const IColumn & column, size_t row_num) const
+ {
+ return has() && value == column[row_num];
+ }
+
+ bool isEqualTo(const Self & to) const
+ {
+ return has() && to.value == value;
+ }
+
+ static bool allocatesMemoryInArena()
+ {
+ return false;
+ }
+};
+#endif
+
+/** What is the difference between the aggregate functions min, max, any, anyLast
+ * (the condition that the stored value is replaced by a new one,
+ * as well as, of course, the name).
+ */
+
+template <typename Data>
+struct AggregateFunctionMinData : Data
+{
+ using Self = AggregateFunctionMinData;
+
+ bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeIfLess(column, row_num, arena); }
+ bool changeIfBetter(const Self & to, Arena * arena) { return this->changeIfLess(to, arena); }
+};
+
+template <typename Data>
+struct AggregateFunctionMaxData : Data
+{
+ using Self = AggregateFunctionMaxData;
+
+ bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeIfGreater(column, row_num, arena); }
+ bool changeIfBetter(const Self & to, Arena * arena) { return this->changeIfGreater(to, arena); }
+};
+
+template <typename Data>
+struct AggregateFunctionAnyData : Data
+{
+ using Self = AggregateFunctionAnyData;
+ static constexpr bool is_any = true;
+
+ bool changeIfBetter(const IColumn & column, size_t row_num, Arena * arena) { return this->changeFirstTime(column, row_num, arena); }
+ bool changeIfBetter(const Self & to, Arena * arena) { return this->changeFirstTime(to, arena); }
+};
+
+template <typename Data>
+class AggregateFunctionsSingleValue final : public IAggregateFunctionDataHelper<Data, AggregateFunctionsSingleValue<Data>>
+{
+ static constexpr bool is_any = Data::is_any;
+
+private:
+ DataTypePtr & type;
+
+public:
+ AggregateFunctionsSingleValue(const DataTypePtr & type_)
+ : IAggregateFunctionDataHelper<Data, AggregateFunctionsSingleValue<Data>>({type_}, {})
+ , type(this->argument_types[0])
+ {
+#if 0
+ if (StringRef(Data::name()) == StringRef("min")
+ || StringRef(Data::name()) == StringRef("max"))
+ {
+ if (!type->isComparable())
+ throw Exception("Illegal type " + type->getName() + " of argument of aggregate function " + getName()
+ + " because the values of that data type are not comparable");
+ }
+#endif
+ }
+
+ DataTypePtr getReturnType() const override
+ {
+ return type;
+ }
+
+ void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
+ {
+ this->data(place).changeIfBetter(*columns[0], row_num, arena);
+ }
+
+ void addBatchSinglePlace(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr place,
+ const IColumn ** columns,
+ Arena * arena,
+ ssize_t if_argument_pos) const override
+ {
+ if constexpr (is_any)
+ if (this->data(place).has())
+ return;
+ if (if_argument_pos >= 0)
+ {
+ const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).raw_values();
+ for (size_t i = row_begin; i < row_end; ++i)
+ {
+ if (flags[i])
+ {
+ this->data(place).changeIfBetter(*columns[0], i, arena);
+ if constexpr (is_any)
+ break;
+ }
+ }
+ }
+ else
+ {
+ for (size_t i = row_begin; i < row_end; ++i)
+ {
+ this->data(place).changeIfBetter(*columns[0], i, arena);
+ if constexpr (is_any)
+ break;
+ }
+ }
+ }
+
+ void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
+ {
+ this->data(place).changeIfBetter(this->data(rhs), arena);
+ }
+
+ bool allocatesMemoryInArena() const override
+ {
+ return Data::allocatesMemoryInArena();
+ }
+
+ void insertResultInto(AggregateDataPtr __restrict place, MutableColumn & to, Arena *) const override
+ {
+ this->data(place).insertResultInto(to);
+ }
+};
+
+template <template <typename> typename AggFunc, template <typename> typename AggData>
+inline std::shared_ptr<IAggregateFunction> createAggregateFunctionSingleValue(const DataTypes & argument_types)
+{
+ if (argument_types.size() != 1)
+ return {};
+
+ const DataTypePtr & argument_type = argument_types[0];
+
+ switch (argument_type->id()) {
+ case arrow::Type::INT8:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::Int8Type>>>>(argument_type);
+ case arrow::Type::INT16:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::Int16Type>>>>(argument_type);
+ case arrow::Type::INT32:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::Int32Type>>>>(argument_type);
+ case arrow::Type::INT64:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::Int64Type>>>>(argument_type);
+ case arrow::Type::UINT8:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::UInt8Type>>>>(argument_type);
+ case arrow::Type::UINT16:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::UInt16Type>>>>(argument_type);
+ case arrow::Type::UINT32:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::UInt32Type>>>>(argument_type);
+ case arrow::Type::UINT64:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::UInt64Type>>>>(argument_type);
+ case arrow::Type::FLOAT:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::FloatType>>>>(argument_type);
+ case arrow::Type::DOUBLE:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::DoubleType>>>>(argument_type);
+ case arrow::Type::TIMESTAMP:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::TimestampType>>>>(argument_type);
+ case arrow::Type::DURATION:
+ return std::make_shared<AggFunc<AggData<SingleValueDataFixed<arrow::DurationType>>>>(argument_type);
+ case arrow::Type::BINARY:
+ return std::make_shared<AggFunc<AggData<SingleValueDataString<false>>>>(argument_type);
+ case arrow::Type::STRING:
+ return std::make_shared<AggFunc<AggData<SingleValueDataString<true>>>>(argument_type);
+ default:
+ break;
+ }
+
+ //return std::make_shared<AggFunc<AggData<SingleValueDataGeneric>>>(argument_type); // TODO
+ return {};
+}
+
+template <template <typename> typename AggFunc, template <typename> typename AggData>
+class WrappedMinMaxAny final : public ArrowAggregateFunctionWrapper
+{
+public:
+ WrappedMinMaxAny(std::string name)
+ : ArrowAggregateFunctionWrapper(std::move(name))
+ {}
+
+ AggregateFunctionPtr getHouseFunction(const DataTypes & argument_types) override
+ {
+ return createAggregateFunctionSingleValue<AggFunc, AggData>(argument_types);
+ }
+};
+
+using WrappedMin = WrappedMinMaxAny<AggregateFunctionsSingleValue, AggregateFunctionMinData>;
+using WrappedMax = WrappedMinMaxAny<AggregateFunctionsSingleValue, AggregateFunctionMaxData>;
+using WrappedAny = WrappedMinMaxAny<AggregateFunctionsSingleValue, AggregateFunctionAnyData>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionSum.h b/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionSum.h
new file mode 100644
index 00000000000..52844b95da2
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/AggregateFunctions/AggregateFunctionSum.h
@@ -0,0 +1,302 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <cstring>
+#include <type_traits>
+
+#include <AggregateFunctions/IAggregateFunction.h>
+
+namespace CH
+{
+
+/// Uses addOverflow method (if available) to avoid UB for sumWithOverflow()
+///
+/// Since NO_SANITIZE_UNDEFINED works only for the function itself, without
+/// callers, and in case of non-POD type (i.e. Decimal) you have overwritten
+/// operator+=(), which will have UB.
+template <typename T>
+struct AggregateFunctionSumAddOverflowImpl
+{
+ static void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(T & lhs, const T & rhs)
+ {
+ lhs += rhs;
+ }
+};
+
+template <typename T>
+struct AggregateFunctionSumData
+{
+ using Impl = AggregateFunctionSumAddOverflowImpl<T>;
+ T sum{};
+
+ void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(T value)
+ {
+ Impl::add(sum, value);
+ }
+
+ template <typename Value>
+ void NO_SANITIZE_UNDEFINED NO_INLINE addManyImpl(const Value * __restrict ptr, size_t start, size_t end) /// NOLINT
+ {
+ ptr += start;
+ size_t count = end - start;
+ const auto * end_ptr = ptr + count;
+
+ if constexpr (std::is_floating_point_v<T>)
+ {
+ /// Compiler cannot unroll this loop, do it manually.
+ /// (at least for floats, most likely due to the lack of -fassociative-math)
+
+ /// Something around the number of SSE registers * the number of elements fit in register.
+ constexpr size_t unroll_count = 128 / sizeof(T);
+ T partial_sums[unroll_count]{};
+
+ const auto * unrolled_end = ptr + (count / unroll_count * unroll_count);
+
+ while (ptr < unrolled_end)
+ {
+ for (size_t i = 0; i < unroll_count; ++i)
+ Impl::add(partial_sums[i], ptr[i]);
+ ptr += unroll_count;
+ }
+
+ for (size_t i = 0; i < unroll_count; ++i)
+ Impl::add(sum, partial_sums[i]);
+ }
+
+ /// clang cannot vectorize the loop if accumulator is class member instead of local variable.
+ T local_sum{};
+ while (ptr < end_ptr)
+ {
+ Impl::add(local_sum, *ptr);
+ ++ptr;
+ }
+ Impl::add(sum, local_sum);
+ }
+
+ /// Vectorized version
+ template <typename Value>
+ void NO_INLINE addMany(const Value * __restrict ptr, size_t start, size_t end)
+ {
+ addManyImpl(ptr, start, end);
+ }
+
+ template <typename Value, bool add_if_zero>
+ void NO_SANITIZE_UNDEFINED NO_INLINE addManyConditionalInternalImpl(
+ const Value * __restrict ptr,
+ const uint8_t * __restrict condition_map,
+ size_t start,
+ size_t end) /// NOLINT
+ {
+ ptr += start;
+ size_t count = end - start;
+ const auto * end_ptr = ptr + count;
+
+ if constexpr (std::is_integral_v<T>)
+ {
+ /// For integers we can vectorize the operation if we replace the null check using a multiplication (by 0 for null, 1 for not null)
+ /// https://quick-bench.com/q/MLTnfTvwC2qZFVeWHfOBR3U7a8I
+ T local_sum{};
+ while (ptr < end_ptr)
+ {
+ T multiplier = !*condition_map == add_if_zero;
+ Impl::add(local_sum, *ptr * multiplier);
+ ++ptr;
+ ++condition_map;
+ }
+ Impl::add(sum, local_sum);
+ return;
+ }
+
+ if constexpr (std::is_floating_point_v<T>)
+ {
+ /// For floating point we use a similar trick as above, except that now we reinterpret the floating point number as an unsigned
+ /// integer of the same size and use a mask instead (0 to discard, 0xFF..FF to keep)
+ static_assert(sizeof(Value) == 4 || sizeof(Value) == 8);
+ using equivalent_integer = typename std::conditional_t<sizeof(Value) == 4, UInt32, UInt64>;
+
+ constexpr size_t unroll_count = 128 / sizeof(T);
+ T partial_sums[unroll_count]{};
+
+ const auto * unrolled_end = ptr + (count / unroll_count * unroll_count);
+
+ while (ptr < unrolled_end)
+ {
+ for (size_t i = 0; i < unroll_count; ++i)
+ {
+ equivalent_integer value;
+ std::memcpy(&value, &ptr[i], sizeof(Value));
+ value &= (!condition_map[i] != add_if_zero) - 1;
+ Value d;
+ std::memcpy(&d, &value, sizeof(Value));
+ Impl::add(partial_sums[i], d);
+ }
+ ptr += unroll_count;
+ condition_map += unroll_count;
+ }
+
+ for (size_t i = 0; i < unroll_count; ++i)
+ Impl::add(sum, partial_sums[i]);
+ }
+
+ T local_sum{};
+ while (ptr < end_ptr)
+ {
+ if (!*condition_map == add_if_zero)
+ Impl::add(local_sum, *ptr);
+ ++ptr;
+ ++condition_map;
+ }
+ Impl::add(sum, local_sum);
+ }
+
+ /// Vectorized version
+ template <typename Value, bool add_if_zero>
+ void NO_INLINE addManyConditionalInternal(const Value * __restrict ptr, const uint8_t * __restrict condition_map, size_t start, size_t end)
+ {
+ addManyConditionalInternalImpl<Value, add_if_zero>(ptr, condition_map, start, end);
+ }
+
+ template <typename Value>
+ void ALWAYS_INLINE addManyNotNull(const Value * __restrict ptr, const uint8_t * __restrict null_map, size_t start, size_t end)
+ {
+ return addManyConditionalInternal<Value, true>(ptr, null_map, start, end);
+ }
+
+ template <typename Value>
+ void ALWAYS_INLINE addManyConditional(const Value * __restrict ptr, const uint8_t * __restrict cond_map, size_t start, size_t end)
+ {
+ return addManyConditionalInternal<Value, false>(ptr, cond_map, start, end);
+ }
+
+ void NO_SANITIZE_UNDEFINED merge(const AggregateFunctionSumData & rhs)
+ {
+ Impl::add(sum, rhs.sum);
+ }
+
+ T get() const
+ {
+ return sum;
+ }
+};
+
+
+/// Counts the sum of the numbers.
+template <typename T, typename TResult, typename Data>
+class AggregateFunctionSum final : public IAggregateFunctionDataHelper<Data, AggregateFunctionSum<T, TResult, Data>>
+{
+public:
+ using ColumnType = arrow::NumericArray<T>;
+ using MutableColumnType = arrow::NumericBuilder<T>;
+
+ explicit AggregateFunctionSum(const DataTypes & argument_types_)
+ : IAggregateFunctionDataHelper<Data, AggregateFunctionSum<T, TResult, Data>>(argument_types_, {})
+ {}
+
+ AggregateFunctionSum(const IDataType & /*data_type*/, const DataTypes & argument_types_)
+ : IAggregateFunctionDataHelper<Data, AggregateFunctionSum<T, TResult, Data>>(argument_types_, {})
+ {}
+
+ DataTypePtr getReturnType() const override
+ {
+ return std::make_shared<TResult>();
+ }
+
+ bool allocatesMemoryInArena() const override { return false; }
+
+ void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
+ {
+ const auto & column = assert_cast<const ColumnType &>(*columns[0]);
+ this->data(place).add(column.Value(row_num));
+ }
+
+ void addBatchSinglePlace(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** columns,
+ Arena *,
+ ssize_t if_argument_pos) const override
+ {
+ const auto & column = assert_cast<const ColumnType &>(*columns[0]);
+ if (if_argument_pos >= 0)
+ {
+ const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).raw_values();
+ this->data(place).addManyConditional(column.raw_values(), flags, row_begin, row_end);
+ }
+ else
+ {
+ this->data(place).addMany(column.raw_values(), row_begin, row_end);
+ }
+ }
+
+ void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
+ {
+ this->data(place).merge(this->data(rhs));
+ }
+
+ void insertResultInto(AggregateDataPtr __restrict place, MutableColumn & to, Arena *) const override
+ {
+ assert_cast<MutableColumnType &>(to).Append(this->data(place).get()).ok();
+ }
+};
+
+class WrappedSum final : public ArrowAggregateFunctionWrapper
+{
+public:
+ template <typename T>
+ using AggregateFunctionSumWithOverflow =
+ AggregateFunctionSum<T, T, AggregateFunctionSumData<typename arrow::TypeTraits<T>::CType>>;
+
+ WrappedSum(std::string name)
+ : ArrowAggregateFunctionWrapper(std::move(name))
+ {}
+
+ AggregateFunctionPtr getHouseFunction(const DataTypes & argument_types) override
+ {
+ return createWithSameType<AggregateFunctionSumWithOverflow>(argument_types);
+ }
+
+ template <template <typename> typename AggFunc>
+ std::shared_ptr<IAggregateFunction> createWithSameType(const DataTypes & argument_types)
+ {
+ if (argument_types.size() != 1)
+ return {};
+
+ const DataTypePtr & type = argument_types[0];
+
+ switch (type->id()) {
+ case arrow::Type::INT8:
+ return std::make_shared<AggFunc<arrow::Int8Type>>(argument_types);
+ case arrow::Type::INT16:
+ return std::make_shared<AggFunc<arrow::Int16Type>>(argument_types);
+ case arrow::Type::INT32:
+ return std::make_shared<AggFunc<arrow::Int32Type>>(argument_types);
+ case arrow::Type::INT64:
+ return std::make_shared<AggFunc<arrow::Int64Type>>(argument_types);
+ case arrow::Type::UINT8:
+ return std::make_shared<AggFunc<arrow::UInt8Type>>(argument_types);
+ case arrow::Type::UINT16:
+ return std::make_shared<AggFunc<arrow::UInt16Type>>(argument_types);
+ case arrow::Type::UINT32:
+ return std::make_shared<AggFunc<arrow::UInt32Type>>(argument_types);
+ case arrow::Type::UINT64:
+ return std::make_shared<AggFunc<arrow::UInt64Type>>(argument_types);
+ case arrow::Type::FLOAT:
+ return std::make_shared<AggFunc<arrow::FloatType>>(argument_types);
+ case arrow::Type::DOUBLE:
+ return std::make_shared<AggFunc<arrow::DoubleType>>(argument_types);
+ case arrow::Type::DURATION:
+ return std::make_shared<AggFunc<arrow::DurationType>>(argument_types);
+ default:
+ break;
+ }
+
+ return {};
+ }
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/AggregateFunctions/IAggregateFunction.h b/ydb/library/arrow_clickhouse/AggregateFunctions/IAggregateFunction.h
new file mode 100644
index 00000000000..81c425d2494
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/AggregateFunctions/IAggregateFunction.h
@@ -0,0 +1,495 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+#include <type_traits>
+
+namespace CH
+{
+
+class Arena;
+class ReadBuffer;
+class WriteBuffer;
+
+using AggregateDataPtr = char *;
+using ConstAggregateDataPtr = const char *;
+
+class IAggregateFunction;
+using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
+struct AggregateFunctionProperties;
+
+/** Aggregate functions interface.
+ * Instances of classes with this interface do not contain the data itself for aggregation,
+ * but contain only metadata (description) of the aggregate function,
+ * as well as methods for creating, deleting and working with data.
+ * The data resulting from the aggregation (intermediate computing states) is stored in other objects
+ * (which can be created in some memory pool),
+ * and IAggregateFunction is the external interface for manipulating them.
+ */
+class IAggregateFunction : public std::enable_shared_from_this<IAggregateFunction>
+{
+public:
+ IAggregateFunction(const DataTypes & argument_types_, const Array & parameters_)
+ : argument_types(argument_types_), parameters(parameters_) {}
+
+ /// Get the result type.
+ virtual DataTypePtr getReturnType() const = 0;
+#if 0
+ /// Get the data type of internal state. By default it is AggregateFunction(name(params), argument_types...).
+ virtual DataTypePtr getStateType() const;
+#endif
+
+ virtual ~IAggregateFunction() = default;
+
+ /** Data manipulating functions. */
+
+ /** Create empty data for aggregation with `placement new` at the specified location.
+ * You will have to destroy them using the `destroy` method.
+ */
+ virtual void create(AggregateDataPtr __restrict place) const = 0;
+
+ /// Delete data for aggregation.
+ virtual void destroy(AggregateDataPtr __restrict place) const noexcept = 0;
+
+ /// It is not necessary to delete data.
+ virtual bool hasTrivialDestructor() const = 0;
+
+ /// Get `sizeof` of structure with data.
+ virtual size_t sizeOfData() const = 0;
+
+ /// How the data structure should be aligned.
+ virtual size_t alignOfData() const = 0;
+
+ /** Adds a value into aggregation data on which place points to.
+ * columns points to columns containing arguments of aggregation function.
+ * row_num is number of row which should be added.
+ * Additional parameter arena should be used instead of standard memory allocator if the addition requires memory allocation.
+ */
+ virtual void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const = 0;
+
+ /// Merges state (on which place points to) with other state of current aggregation function.
+ virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const = 0;
+#if 0
+ /// Serializes state (to transmit it over the network, for example).
+ virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const = 0;
+
+ /// Deserializes state. This function is called only for empty (just created) states.
+ virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const = 0;
+#endif
+ /// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()).
+ virtual bool allocatesMemoryInArena() const = 0;
+
+ /// Inserts results into a column. This method might modify the state (e.g.
+ /// sort an array), so must be called once, from single thread. The state
+ /// must remain valid though, and the subsequent calls to add/merge/
+ /// insertResultInto must work correctly. This kind of call sequence occurs
+ /// in `runningAccumulate`, or when calculating an aggregate function as a
+ /// window function.
+ virtual void insertResultInto(AggregateDataPtr __restrict place, MutableColumn & to, Arena * arena) const = 0;
+
+ /** Returns true for aggregate functions of type -State
+ * They are executed as other aggregate functions, but not finalized (return an aggregation state that can be combined with another).
+ * Also returns true when the final value of this aggregate function contains State of other aggregate function inside.
+ */
+ virtual bool isState() const { return false; }
+
+ /** The inner loop that uses the function pointer is better than using the virtual function.
+ * The reason is that in the case of virtual functions GCC 5.1.2 generates code,
+ * which, at each iteration of the loop, reloads the function address (the offset value in the virtual function table) from memory to the register.
+ * This gives a performance drop on simple queries around 12%.
+ * After the appearance of better compilers, the code can be removed.
+ */
+ using AddFunc = void (*)(const IAggregateFunction *, AggregateDataPtr, const IColumn **, size_t, Arena *);
+ virtual AddFunc getAddressOfAddFunction() const = 0;
+
+ /** Contains a loop with calls to "add" function. You can collect arguments into array "places"
+ * and do a single call to "addBatch" for devirtualization and inlining.
+ */
+ virtual void addBatch( /// NOLINT
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset,
+ const IColumn ** columns,
+ Arena * arena,
+ ssize_t if_argument_pos = -1) const = 0;
+
+ virtual void mergeBatch(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset,
+ const AggregateDataPtr * rhs,
+ Arena * arena) const = 0;
+
+ /** The same for single place.
+ */
+ virtual void addBatchSinglePlace( /// NOLINT
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** columns,
+ Arena * arena,
+ ssize_t if_argument_pos = -1) const = 0;
+
+ /** The case when the aggregation key is UInt8
+ * and pointers to aggregation states are stored in AggregateDataPtr[256] lookup table.
+ */
+ virtual void addBatchLookupTable8(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset,
+ std::function<void(AggregateDataPtr &)> init,
+ const UInt8 * key,
+ const IColumn ** columns,
+ Arena * arena) const = 0;
+
+ /** Insert result of aggregate function into result column with batch size.
+ * If destroy_place_after_insert is true. Then implementation of this method
+ * must destroy aggregate place if insert state into result column was successful.
+ * All places that were not inserted must be destroyed if there was exception during insert into result column.
+ */
+ virtual void insertResultIntoBatch(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset,
+ MutableColumn & to,
+ Arena * arena,
+ bool destroy_place_after_insert) const = 0;
+
+ /** Destroy batch of aggregate places.
+ */
+ virtual void destroyBatch(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset) const noexcept = 0;
+
+ const DataTypes & getArgumentTypes() const { return argument_types; }
+ const Array & getParameters() const { return parameters; }
+
+protected:
+ DataTypes argument_types;
+ Array parameters;
+};
+
+
+/// Implement method to obtain an address of 'add' function.
+template <typename Derived>
+class IAggregateFunctionHelper : public IAggregateFunction
+{
+private:
+ static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena)
+ {
+ static_cast<const Derived &>(*that).add(place, columns, row_num, arena);
+ }
+
+public:
+ IAggregateFunctionHelper(const DataTypes & argument_types_, const Array & parameters_)
+ : IAggregateFunction(argument_types_, parameters_) {}
+
+ AddFunc getAddressOfAddFunction() const override { return &addFree; }
+
+ void addBatch( /// NOLINT
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset,
+ const IColumn ** columns,
+ Arena * arena,
+ ssize_t if_argument_pos = -1) const override
+ {
+ if (if_argument_pos >= 0)
+ {
+ const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).raw_values();
+ for (size_t i = row_begin; i < row_end; ++i)
+ {
+ if (flags[i] && places[i])
+ static_cast<const Derived *>(this)->add(places[i] + place_offset, columns, i, arena);
+ }
+ }
+ else
+ {
+ for (size_t i = row_begin; i < row_end; ++i)
+ if (places[i])
+ static_cast<const Derived *>(this)->add(places[i] + place_offset, columns, i, arena);
+ }
+ }
+
+ void mergeBatch(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset,
+ const AggregateDataPtr * rhs,
+ Arena * arena) const override
+ {
+ for (size_t i = row_begin; i < row_end; ++i)
+ if (places[i])
+ static_cast<const Derived *>(this)->merge(places[i] + place_offset, rhs[i], arena);
+ }
+
+ void addBatchSinglePlace( /// NOLINT
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr __restrict place,
+ const IColumn ** columns,
+ Arena * arena,
+ ssize_t if_argument_pos = -1) const override
+ {
+ if (if_argument_pos >= 0)
+ {
+ const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).raw_values();
+ for (size_t i = row_begin; i < row_end; ++i)
+ {
+ if (flags[i])
+ static_cast<const Derived *>(this)->add(place, columns, i, arena);
+ }
+ }
+ else
+ {
+ for (size_t i = row_begin; i < row_end; ++i)
+ static_cast<const Derived *>(this)->add(place, columns, i, arena);
+ }
+ }
+
+ void addBatchLookupTable8(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * map,
+ size_t place_offset,
+ std::function<void(AggregateDataPtr &)> init,
+ const UInt8 * key,
+ const IColumn ** columns,
+ Arena * arena) const override
+ {
+ static constexpr size_t UNROLL_COUNT = 8;
+
+ size_t i = row_begin;
+
+ size_t size_unrolled = (row_end - row_begin) / UNROLL_COUNT * UNROLL_COUNT;
+ for (; i < size_unrolled; i += UNROLL_COUNT)
+ {
+ AggregateDataPtr places[UNROLL_COUNT];
+ for (size_t j = 0; j < UNROLL_COUNT; ++j)
+ {
+ AggregateDataPtr & place = map[key[i + j]];
+ if (unlikely(!place))
+ init(place);
+
+ places[j] = place;
+ }
+
+ for (size_t j = 0; j < UNROLL_COUNT; ++j)
+ static_cast<const Derived *>(this)->add(places[j] + place_offset, columns, i + j, arena);
+ }
+
+ for (; i < row_end; ++i)
+ {
+ AggregateDataPtr & place = map[key[i]];
+ if (unlikely(!place))
+ init(place);
+ static_cast<const Derived *>(this)->add(place + place_offset, columns, i, arena);
+ }
+ }
+
+ void insertResultIntoBatch(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset,
+ MutableColumn & to,
+ Arena * arena,
+ bool destroy_place_after_insert) const override
+ {
+ size_t batch_index = row_begin;
+
+ try
+ {
+ for (; batch_index < row_end; ++batch_index)
+ {
+ static_cast<const Derived *>(this)->insertResultInto(places[batch_index] + place_offset, to, arena);
+
+ if (destroy_place_after_insert)
+ static_cast<const Derived *>(this)->destroy(places[batch_index] + place_offset);
+ }
+ }
+ catch (...)
+ {
+ for (size_t destroy_index = batch_index; destroy_index < row_end; ++destroy_index)
+ static_cast<const Derived *>(this)->destroy(places[destroy_index] + place_offset);
+
+ throw;
+ }
+ }
+
+ void destroyBatch(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * places,
+ size_t place_offset) const noexcept override
+ {
+ for (size_t i = row_begin; i < row_end; ++i)
+ {
+ static_cast<const Derived *>(this)->destroy(places[i] + place_offset);
+ }
+ }
+};
+
+
+/// Implements several methods for manipulation with data. T - type of structure with data for aggregation.
+template <typename T, typename Derived>
+class IAggregateFunctionDataHelper : public IAggregateFunctionHelper<Derived>
+{
+protected:
+ using Data = T;
+
+ static Data & data(AggregateDataPtr __restrict place) { return *reinterpret_cast<Data *>(place); }
+ static const Data & data(ConstAggregateDataPtr __restrict place) { return *reinterpret_cast<const Data *>(place); }
+
+public:
+ // Derived class can `override` this to flag that DateTime64 is not supported.
+ static constexpr bool DateTime64Supported = true;
+
+ IAggregateFunctionDataHelper(const DataTypes & argument_types_, const Array & parameters_)
+ : IAggregateFunctionHelper<Derived>(argument_types_, parameters_) {}
+
+ void create(AggregateDataPtr __restrict place) const override /// NOLINT
+ {
+ new (place) Data;
+ }
+
+ void destroy(AggregateDataPtr __restrict place) const noexcept override
+ {
+ data(place).~Data();
+ }
+
+ bool hasTrivialDestructor() const override
+ {
+ return std::is_trivially_destructible_v<Data>;
+ }
+
+ size_t sizeOfData() const override
+ {
+ return sizeof(Data);
+ }
+
+ size_t alignOfData() const override
+ {
+ return alignof(Data);
+ }
+
+ void addBatchLookupTable8(
+ size_t row_begin,
+ size_t row_end,
+ AggregateDataPtr * map,
+ size_t place_offset,
+ std::function<void(AggregateDataPtr &)> init,
+ const UInt8 * key,
+ const IColumn ** columns,
+ Arena * arena) const override
+ {
+ const Derived & func = *static_cast<const Derived *>(this);
+
+ /// If the function is complex or too large, use more generic algorithm.
+
+ if (func.allocatesMemoryInArena() || sizeof(Data) > 16 || func.sizeOfData() != sizeof(Data))
+ {
+ IAggregateFunctionHelper<Derived>::addBatchLookupTable8(row_begin, row_end, map, place_offset, init, key, columns, arena);
+ return;
+ }
+
+ /// Will use UNROLL_COUNT number of lookup tables.
+
+ static constexpr size_t UNROLL_COUNT = 4;
+
+ std::unique_ptr<Data[]> places{new Data[256 * UNROLL_COUNT]};
+ bool has_data[256 * UNROLL_COUNT]{}; /// Separate flags array to avoid heavy initialization.
+
+ size_t i = row_begin;
+
+ /// Aggregate data into different lookup tables.
+
+ size_t size_unrolled = (row_end - row_begin) / UNROLL_COUNT * UNROLL_COUNT;
+ for (; i < size_unrolled; i += UNROLL_COUNT)
+ {
+ for (size_t j = 0; j < UNROLL_COUNT; ++j)
+ {
+ size_t idx = j * 256 + key[i + j];
+ if (unlikely(!has_data[idx]))
+ {
+ new (&places[idx]) Data;
+ has_data[idx] = true;
+ }
+ func.add(reinterpret_cast<char *>(&places[idx]), columns, i + j, nullptr);
+ }
+ }
+
+ /// Merge data from every lookup table to the final destination.
+
+ for (size_t k = 0; k < 256; ++k)
+ {
+ for (size_t j = 0; j < UNROLL_COUNT; ++j)
+ {
+ size_t idx = j * 256 + k;
+ if (has_data[idx])
+ {
+ AggregateDataPtr & place = map[k];
+ if (unlikely(!place))
+ init(place);
+
+ func.merge(place + place_offset, reinterpret_cast<const char *>(&places[idx]), nullptr);
+ }
+ }
+ }
+
+ /// Process tails and add directly to the final destination.
+
+ for (; i < row_end; ++i)
+ {
+ size_t k = key[i];
+ AggregateDataPtr & place = map[k];
+ if (unlikely(!place))
+ init(place);
+
+ func.add(place + place_offset, columns, i, nullptr);
+ }
+ }
+};
+
+
+/// Properties of aggregate function that are independent of argument types and parameters.
+struct AggregateFunctionProperties
+{
+ /** When the function is wrapped with Null combinator,
+ * should we return Nullable type with NULL when no values were aggregated
+ * or we should return non-Nullable type with default value (example: count, countDistinct).
+ */
+ bool returns_default_when_only_null = false;
+
+ /** Result varies depending on the data order (example: groupArray).
+ * Some may also name this property as "non-commutative".
+ */
+ bool is_order_dependent = false;
+};
+
+
+class ArrowAggregateFunctionWrapper : public arrow::compute::ScalarAggregateFunction
+{
+public:
+ ArrowAggregateFunctionWrapper(std::string name)
+ : arrow::compute::ScalarAggregateFunction(std::move(name), arrow::compute::Arity::Unary(), nullptr)
+ {}
+
+ virtual AggregateFunctionPtr getHouseFunction(const DataTypes & argument_types) = 0;
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/AggregationCommon.h b/ydb/library/arrow_clickhouse/AggregationCommon.h
new file mode 100644
index 00000000000..ecd475eacca
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/AggregationCommon.h
@@ -0,0 +1,337 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <array>
+
+#include <Columns/ColumnsCommon.h>
+#include <Common/HashTable/Hash.h>
+#include <Common/memcpySmall.h>
+
+#include <common/StringRef.h>
+
+#if defined(__SSSE3__) && !defined(MEMORY_SANITIZER)
+#include <tmmintrin.h>
+#endif
+
+
+namespace CH
+{
+
+/// When packing the values of nullable columns at a given row, we have to
+/// store the fact that these values are nullable or not. This is achieved
+/// by encoding this information as a bitmap. Let S be the size in bytes of
+/// a packed values binary blob and T the number of bytes we may place into
+/// this blob, the size that the bitmap shall occupy in the blob is equal to:
+/// ceil(T/8). Thus we must have: S = T + ceil(T/8). Below we indicate for
+/// each value of S, the corresponding value of T, and the bitmap size:
+///
+/// 32,28,4
+/// 16,14,2
+/// 8,7,1
+/// 4,3,1
+/// 2,1,1
+///
+
+namespace
+{
+
+template <typename T>
+constexpr auto getBitmapSize()
+{
+ return
+ (sizeof(T) == 32) ?
+ 4 :
+ (sizeof(T) == 16) ?
+ 2 :
+ ((sizeof(T) == 8) ?
+ 1 :
+ ((sizeof(T) == 4) ?
+ 1 :
+ ((sizeof(T) == 2) ?
+ 1 :
+ 0)));
+}
+
+}
+
+template<typename T, size_t step>
+void fillFixedBatch(size_t num_rows, const T * source, T * dest)
+{
+ for (size_t i = 0; i < num_rows; ++i)
+ {
+ *dest = *source;
+ ++source;
+ dest += step;
+ }
+}
+
+/// Move keys of size T into binary blob, starting from offset.
+/// It is assumed that offset is aligned to sizeof(T).
+/// Example: sizeof(key) = 16, sizeof(T) = 4, offset = 8
+/// out[0] : [--------****----]
+/// out[1] : [--------****----]
+/// ...
+template<typename T, typename Key>
+void fillFixedBatch(size_t keys_size, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, PaddedPODArray<Key> & out, size_t & offset)
+{
+ for (size_t i = 0; i < keys_size; ++i)
+ {
+ if (key_sizes[i] == sizeof(T))
+ {
+ const auto * column = key_columns[i];
+ size_t num_rows = column->length();
+ out.resize_fill(num_rows);
+#if 0
+ /// Note: here we violate strict aliasing.
+ /// It should be ok as log as we do not reffer to any value from `out` before filling.
+ const char * source = assert_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<sizeof(T)>();
+ T * dest = reinterpret_cast<T *>(reinterpret_cast<char *>(out.data()) + offset);
+ fillFixedBatch<T, sizeof(Key) / sizeof(T)>(num_rows, reinterpret_cast<const T *>(source), dest);
+ offset += sizeof(T);
+#else
+ T * dest = reinterpret_cast<T *>(reinterpret_cast<char *>(out.data()) + offset);
+ switch (sizeof(T))
+ {
+ case 1:
+ case 2:
+ case 4:
+ case 8:
+ {
+ const uint8_t * source = assert_cast<const ColumnUInt8 *>(column)->raw_values();
+ fillFixedBatch<T, sizeof(Key) / sizeof(T)>(num_rows, reinterpret_cast<const T *>(source), dest);
+ break;
+ }
+ default:
+ {
+ const uint8_t * source = assert_cast<const ColumnFixedString *>(column)->raw_values();
+ fillFixedBatch<T, sizeof(Key) / sizeof(T)>(num_rows, reinterpret_cast<const T *>(source), dest);
+ break;
+ }
+ }
+ offset += sizeof(T);
+#endif
+ }
+ }
+}
+
+/// Pack into a binary blob of type T a set of fixed-size keys. Granted that all the keys fit into the
+/// binary blob. Keys are placed starting from the longest one.
+template <typename T>
+void packFixedBatch(size_t keys_size, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, PaddedPODArray<T> & out)
+{
+ size_t offset = 0;
+ fillFixedBatch<UInt128>(keys_size, key_columns, key_sizes, out, offset);
+ fillFixedBatch<UInt64>(keys_size, key_columns, key_sizes, out, offset);
+ fillFixedBatch<UInt32>(keys_size, key_columns, key_sizes, out, offset);
+ fillFixedBatch<UInt16>(keys_size, key_columns, key_sizes, out, offset);
+ fillFixedBatch<UInt8>(keys_size, key_columns, key_sizes, out, offset);
+}
+
+template <typename T>
+using KeysNullMap = std::array<UInt8, getBitmapSize<T>()>;
+
+/// Pack into a binary blob of type T a set of fixed-size keys. Granted that all the keys fit into the
+/// binary blob, they are disposed in it consecutively.
+template <typename T>
+static inline T ALWAYS_INLINE packFixed(
+ size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, const Sizes & key_sizes)
+{
+ T key{};
+ char * bytes = reinterpret_cast<char *>(&key);
+ size_t offset = 0;
+
+ for (size_t j = 0; j < keys_size; ++j)
+ {
+ size_t index = i;
+ const IColumn * column = key_columns[j];
+
+ switch (key_sizes[j])
+ {
+ case 1:
+ {
+ memcpy(bytes + offset, assert_cast<const ColumnUInt8 *>(column)->raw_values() + index, 1);
+ offset += 1;
+ }
+ break;
+ case 2:
+ if constexpr (sizeof(T) >= 2) /// To avoid warning about memcpy exceeding object size.
+ {
+ memcpy(bytes + offset, assert_cast<const ColumnUInt16 *>(column)->raw_values() + index, 2);
+ offset += 2;
+ }
+ break;
+ case 4:
+ if constexpr (sizeof(T) >= 4)
+ {
+ memcpy(bytes + offset, assert_cast<const ColumnUInt32 *>(column)->raw_values() + index, 4);
+ offset += 4;
+ }
+ break;
+ case 8:
+ if constexpr (sizeof(T) >= 8)
+ {
+ memcpy(bytes + offset, assert_cast<const ColumnUInt64 *>(column)->raw_values() + index, 8);
+ offset += 8;
+ }
+ break;
+ default:
+ memcpy(bytes + offset, assert_cast<const ColumnFixedString *>(column)->raw_values() + index * key_sizes[j], key_sizes[j]);
+ offset += key_sizes[j];
+ }
+ }
+
+ return key;
+}
+
+/// Similar as above but supports nullable values.
+template <typename T>
+static inline T ALWAYS_INLINE packFixed(
+ size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
+ const KeysNullMap<T> & bitmap)
+{
+ union
+ {
+ T key;
+ char bytes[sizeof(key)] = {};
+ };
+
+ size_t offset = 0;
+
+ static constexpr auto bitmap_size = std::tuple_size<KeysNullMap<T>>::value;
+ static constexpr bool has_bitmap = bitmap_size > 0;
+
+ if (has_bitmap)
+ {
+ memcpy(bytes + offset, bitmap.data(), bitmap_size * sizeof(UInt8));
+ offset += bitmap_size;
+ }
+
+ for (size_t j = 0; j < keys_size; ++j)
+ {
+ bool is_null;
+
+ if (!has_bitmap)
+ is_null = false;
+ else
+ {
+ size_t bucket = j / 8;
+ size_t off = j % 8;
+ is_null = ((bitmap[bucket] >> off) & 1) == 1;
+ }
+
+ if (is_null)
+ continue;
+
+ switch (key_sizes[j])
+ {
+ case 1:
+ memcpy(bytes + offset, assert_cast<const ColumnUInt8 *>(key_columns[j])->raw_values() + i, 1);
+ offset += 1;
+ break;
+ case 2:
+ memcpy(bytes + offset, assert_cast<const ColumnUInt16 *>(key_columns[j])->raw_values() + i, 2);
+ offset += 2;
+ break;
+ case 4:
+ memcpy(bytes + offset, assert_cast<const ColumnUInt32 *>(key_columns[j])->raw_values() + i, 4);
+ offset += 4;
+ break;
+ case 8:
+ memcpy(bytes + offset, assert_cast<const ColumnUInt64 *>(key_columns[j])->raw_values() + i, 8);
+ offset += 8;
+ break;
+ default:
+ memcpy(bytes + offset, assert_cast<const ColumnFixedString *>(key_columns[j])->raw_values() + i * key_sizes[j], key_sizes[j]);
+ offset += key_sizes[j];
+ }
+ }
+
+ return key;
+}
+
+
+/// Hash a set of keys into a UInt128 value.
+static inline UInt128 ALWAYS_INLINE hash128(size_t row, size_t keys_size, const ColumnRawPtrs & key_columns)
+{
+ UInt128 key;
+ SipHash hash;
+
+ for (size_t j = 0; j < keys_size; ++j)
+ updateHashWithValue(*key_columns[j], row, hash);
+
+ hash.get128(key);
+ return key;
+}
+
+
+/// Copy keys to the pool. Then put into pool StringRefs to them and return the pointer to the first.
+static inline StringRef * ALWAYS_INLINE placeKeysInPool(
+ size_t keys_size, StringRefs & keys, Arena & pool)
+{
+ for (size_t j = 0; j < keys_size; ++j)
+ {
+ char * place = pool.alloc(keys[j].size);
+ memcpySmallAllowReadWriteOverflow15(place, keys[j].data, keys[j].size);
+ keys[j].data = place;
+ }
+
+ /// Place the StringRefs on the newly copied keys in the pool.
+ char * res = pool.alignedAlloc(keys_size * sizeof(StringRef), alignof(StringRef));
+ memcpySmallAllowReadWriteOverflow15(res, keys.data(), keys_size * sizeof(StringRef));
+
+ return reinterpret_cast<StringRef *>(res);
+}
+
+
+/** Serialize keys into a continuous chunk of memory.
+ */
+static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous(
+ size_t row, size_t keys_size, const ColumnRawPtrs & key_columns, Arena & pool)
+{
+ const char * begin = nullptr;
+
+ size_t sum_size = 0;
+ for (size_t j = 0; j < keys_size; ++j)
+ sum_size += serializeValueIntoArena(*key_columns[j], row, pool, begin).size;
+
+ return {begin, sum_size};
+}
+
+
+/** Pack elements with shuffle instruction.
+ * See the explanation in ColumnsHashing.h
+ */
+#if defined(__SSSE3__) && !defined(MEMORY_SANITIZER)
+template <typename T>
+static T inline packFixedShuffle(
+ const char * __restrict * __restrict srcs,
+ size_t num_srcs,
+ const size_t * __restrict elem_sizes,
+ size_t idx,
+ const uint8_t * __restrict masks)
+{
+ assert(num_srcs > 0);
+
+ __m128i res = _mm_shuffle_epi8(
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(srcs[0] + elem_sizes[0] * idx)),
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(masks)));
+
+ for (size_t i = 1; i < num_srcs; ++i)
+ {
+ res = _mm_xor_si128(res,
+ _mm_shuffle_epi8(
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(srcs[i] + elem_sizes[i] * idx)),
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(&masks[i * sizeof(T)]))));
+ }
+
+ T out;
+ __builtin_memcpy(&out, &res, sizeof(T));
+ return out;
+}
+#endif
+
+}
diff --git a/ydb/library/arrow_clickhouse/Aggregator.cpp b/ydb/library/arrow_clickhouse/Aggregator.cpp
new file mode 100644
index 00000000000..5a6a89befe9
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Aggregator.cpp
@@ -0,0 +1,1554 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#include "Aggregator.h"
+#include <DataStreams/IBlockInputStream.h>
+
+
+namespace CH
+{
+
+AggregatedDataVariants::~AggregatedDataVariants()
+{
+ if (aggregator && !aggregator->all_aggregates_has_trivial_destructor)
+ {
+ try
+ {
+ aggregator->destroyAllAggregateStates(*this);
+ }
+ catch (...)
+ {
+ //tryLogCurrentException(__PRETTY_FUNCTION__);
+ }
+ }
+}
+
+Header Aggregator::getHeader(bool final) const
+{
+ return params.getHeader(final);
+}
+
+Header Aggregator::Params::getHeader(
+ const Header & src_header,
+ const Header & intermediate_header,
+ const ColumnNumbers & keys,
+ const AggregateDescriptions & aggregates,
+ bool final)
+{
+ ColumnsWithTypeAndName fields;
+ if (intermediate_header)
+ {
+ fields = intermediate_header->fields();
+
+ if (final)
+ {
+ for (const auto & aggregate : aggregates)
+ {
+ int agg_pos = intermediate_header->GetFieldIndex(aggregate.column_name);
+ DataTypePtr type = aggregate.function->getReturnType();
+
+ fields[agg_pos] = std::make_shared<ColumnWithTypeAndName>(aggregate.column_name, type);
+ }
+ }
+ }
+ else
+ {
+ fields.reserve(keys.size() + aggregates.size());
+
+ for (const auto & key : keys)
+ fields.push_back(src_header->field(key));
+
+ for (const auto & aggregate : aggregates)
+ {
+ size_t arguments_size = aggregate.arguments.size();
+ DataTypes argument_types(arguments_size);
+ for (size_t j = 0; j < arguments_size; ++j)
+ argument_types[j] = src_header->field(aggregate.arguments[j])->type();
+
+ DataTypePtr type;
+ if (final)
+ type = aggregate.function->getReturnType();
+ else
+ type = std::make_shared<DataTypeAggregateFunction>(
+ aggregate.function, argument_types, aggregate.parameters);
+
+ fields.emplace_back(std::make_shared<ColumnWithTypeAndName>(aggregate.column_name, type));
+ }
+ }
+ return std::make_shared<arrow::Schema>(fields);
+}
+
+
+Aggregator::Aggregator(const Params & params_)
+ : params(params_)
+{
+ aggregate_functions.resize(params.aggregates_size);
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i] = params.aggregates[i].function.get();
+
+ /// Initialize sizes of aggregation states and its offsets.
+ offsets_of_aggregate_states.resize(params.aggregates_size);
+ total_size_of_aggregate_states = 0;
+ all_aggregates_has_trivial_destructor = true;
+
+ // aggregate_states will be aligned as below:
+ // |<-- state_1 -->|<-- pad_1 -->|<-- state_2 -->|<-- pad_2 -->| .....
+ //
+ // pad_N will be used to match alignment requirement for each next state.
+ // The address of state_1 is aligned based on maximum alignment requirements in states
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ {
+ offsets_of_aggregate_states[i] = total_size_of_aggregate_states;
+
+ total_size_of_aggregate_states += params.aggregates[i].function->sizeOfData();
+
+ // aggregate states are aligned based on maximum requirement
+ align_aggregate_states = std::max(align_aggregate_states, params.aggregates[i].function->alignOfData());
+
+ // If not the last aggregate_state, we need pad it so that next aggregate_state will be aligned.
+ if (i + 1 < params.aggregates_size)
+ {
+ size_t alignment_of_next_state = params.aggregates[i + 1].function->alignOfData();
+ if ((alignment_of_next_state & (alignment_of_next_state - 1)) != 0)
+ throw Exception("Logical error: alignOfData is not 2^N");
+
+ /// Extend total_size to next alignment requirement
+ /// Add padding by rounding up 'total_size_of_aggregate_states' to be a multiplier of alignment_of_next_state.
+ total_size_of_aggregate_states = (total_size_of_aggregate_states + alignment_of_next_state - 1) / alignment_of_next_state * alignment_of_next_state;
+ }
+
+ if (!params.aggregates[i].function->hasTrivialDestructor())
+ all_aggregates_has_trivial_destructor = false;
+ }
+
+ method_chosen = chooseAggregationMethod();
+ HashMethodContext::Settings cache_settings;
+ cache_settings.max_threads = params.max_threads;
+ aggregation_state_cache = AggregatedDataVariants::createCache(method_chosen, cache_settings);
+}
+
+
+AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
+{
+ /// If no keys. All aggregating to single row.
+ if (params.keys_size == 0)
+ return AggregatedDataVariants::Type::without_key;
+
+ auto& header = (params.src_header ? params.src_header : params.intermediate_header);
+
+ DataTypes types;
+ types.reserve(params.keys_size);
+ for (const auto & pos : params.keys)
+ types.push_back(header->field(pos)->type());
+
+ size_t keys_bytes = 0;
+ size_t num_fixed_contiguous_keys = 0;
+
+ key_sizes.resize(params.keys_size);
+ for (size_t j = 0; j < params.keys.size(); ++j)
+ {
+ if (size_t fixed_size = fixedContiguousSize(types[j]))
+ {
+ ++num_fixed_contiguous_keys;
+ key_sizes[j] = fixed_size;
+ keys_bytes += fixed_size;
+ }
+ }
+
+ //if (has_nullable_key)
+ {
+ if (params.keys_size == num_fixed_contiguous_keys)
+ {
+ /// Pack if possible all the keys along with information about which key values are nulls
+ /// into a fixed 16- or 32-byte blob.
+ if (std::tuple_size<KeysNullMap<UInt128>>::value + keys_bytes <= 16)
+ return AggregatedDataVariants::Type::nullable_keys128;
+ if (std::tuple_size<KeysNullMap<UInt256>>::value + keys_bytes <= 32)
+ return AggregatedDataVariants::Type::nullable_keys256;
+ }
+
+ /// Fallback case.
+ return AggregatedDataVariants::Type::serialized;
+ }
+
+#if 0 // TODO: keys with explicit NOT NULL
+ /// No key has been found to be nullable.
+
+ /// Single numeric key.
+ if (params.keys_size == 1 && types[0]->isValueRepresentedByNumber())
+ {
+ size_t size_of_field = types[0]->getSizeOfValueInMemory();
+
+ if (size_of_field == 1)
+ return AggregatedDataVariants::Type::key8;
+ if (size_of_field == 2)
+ return AggregatedDataVariants::Type::key16;
+ if (size_of_field == 4)
+ return AggregatedDataVariants::Type::key32;
+ if (size_of_field == 8)
+ return AggregatedDataVariants::Type::key64;
+ if (size_of_field == 16)
+ return AggregatedDataVariants::Type::keys128;
+ if (size_of_field == 32)
+ return AggregatedDataVariants::Type::keys256;
+ throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
+ }
+
+ if (params.keys_size == 1 && isFixedString(types[0]))
+ {
+ return AggregatedDataVariants::Type::key_fixed_string;
+ }
+
+ /// If all keys fits in N bits, will use hash table with all keys packed (placed contiguously) to single N-bit key.
+ if (params.keys_size == num_fixed_contiguous_keys)
+ {
+ if (keys_bytes <= 2)
+ return AggregatedDataVariants::Type::keys16;
+ if (keys_bytes <= 4)
+ return AggregatedDataVariants::Type::keys32;
+ if (keys_bytes <= 8)
+ return AggregatedDataVariants::Type::keys64;
+ if (keys_bytes <= 16)
+ return AggregatedDataVariants::Type::keys128;
+ if (keys_bytes <= 32)
+ return AggregatedDataVariants::Type::keys256;
+ }
+
+ /// If single string key - will use hash table with references to it. Strings itself are stored separately in Arena.
+ if (params.keys_size == 1 && isString(types[0]))
+ {
+ return AggregatedDataVariants::Type::key_string;
+ }
+
+ return AggregatedDataVariants::Type::serialized;
+#endif
+}
+
+void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const
+{
+ for (size_t j = 0; j < params.aggregates_size; ++j)
+ {
+ try
+ {
+ /** An exception may occur if there is a shortage of memory.
+ * In order that then everything is properly destroyed, we "roll back" some of the created states.
+ * The code is not very convenient.
+ */
+ aggregate_functions[j]->create(aggregate_data + offsets_of_aggregate_states[j]);
+ }
+ catch (...)
+ {
+ for (size_t rollback_j = 0; rollback_j < j; ++rollback_j)
+ {
+ aggregate_functions[rollback_j]->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]);
+ }
+
+ throw;
+ }
+ }
+}
+
+/** It's interesting - if you remove `noinline`, then gcc for some reason will inline this function, and the performance decreases (~ 10%).
+ * (Probably because after the inline of this function, more internal functions no longer be inlined.)
+ * Inline does not make sense, since the inner loop is entirely inside this function.
+ */
+template <typename Method>
+void NO_INLINE Aggregator::executeImpl(
+ Method & method,
+ Arena * aggregates_pool,
+ size_t row_begin,
+ size_t row_end,
+ ColumnRawPtrs & key_columns,
+ AggregateFunctionInstruction * aggregate_instructions,
+ bool no_more_keys,
+ AggregateDataPtr overflow_row) const
+{
+ typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
+
+ if (!no_more_keys)
+ {
+ executeImplBatch<false>(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
+ }
+ else
+ {
+ executeImplBatch<true>(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row);
+ }
+}
+
+template <bool no_more_keys, typename Method>
+void NO_INLINE Aggregator::executeImplBatch(
+ Method & method,
+ typename Method::State & state,
+ Arena * aggregates_pool,
+ size_t row_begin,
+ size_t row_end,
+ AggregateFunctionInstruction * aggregate_instructions,
+ AggregateDataPtr overflow_row) const
+{
+ /// Optimization for special case when there are no aggregate functions.
+ if (params.aggregates_size == 0)
+ {
+ if constexpr (no_more_keys)
+ return;
+
+ /// For all rows.
+ AggregateDataPtr place = aggregates_pool->alloc(0);
+ for (size_t i = row_begin; i < row_end; ++i)
+ state.emplaceKey(method.data, i, *aggregates_pool).setMapped(place);
+ return;
+ }
+
+ /// Optimization for special case when aggregating by 8bit key.
+ if constexpr (!no_more_keys && std::is_same_v<Method, typename decltype(AggregatedDataVariants::key8)::element_type>)
+ {
+ //if (!has_arrays && !hasSparseArguments(aggregate_instructions))
+ {
+ for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
+ {
+ inst->batch_that->addBatchLookupTable8(
+ row_begin,
+ row_end,
+ reinterpret_cast<AggregateDataPtr *>(method.data.data()),
+ inst->state_offset,
+ [&](AggregateDataPtr & aggregate_data)
+ {
+ aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
+ createAggregateStates(aggregate_data);
+ },
+ state.getKeyData(),
+ inst->batch_arguments,
+ aggregates_pool);
+ }
+ return;
+ }
+ }
+
+ /// NOTE: only row_end-row_start is required, but:
+ /// - this affects only optimize_aggregation_in_order,
+ /// - this is just a pointer, so it should not be significant,
+ /// - and plus this will require other changes in the interface.
+ std::unique_ptr<AggregateDataPtr[]> places(new AggregateDataPtr[row_end]);
+
+ /// For all rows.
+ for (size_t i = row_begin; i < row_end; ++i)
+ {
+ AggregateDataPtr aggregate_data = nullptr;
+
+ if constexpr (!no_more_keys)
+ {
+ auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool);
+
+ /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
+ if (emplace_result.isInserted())
+ {
+ /// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
+ emplace_result.setMapped(nullptr);
+
+ aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
+
+ {
+ createAggregateStates(aggregate_data);
+ }
+
+ emplace_result.setMapped(aggregate_data);
+ }
+ else
+ aggregate_data = emplace_result.getMapped();
+
+ assert(aggregate_data != nullptr);
+ }
+ else
+ {
+ /// Add only if the key already exists.
+ auto find_result = state.findKey(method.data, i, *aggregates_pool);
+ if (find_result.isFound())
+ aggregate_data = find_result.getMapped();
+ else
+ aggregate_data = overflow_row;
+ }
+
+ places[i] = aggregate_data;
+ }
+
+ /// Add values to the aggregate functions.
+ for (size_t i = 0; i < aggregate_functions.size(); ++i)
+ {
+ AggregateFunctionInstruction * inst = aggregate_instructions + i;
+
+ inst->batch_that->addBatch(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool);
+ }
+}
+
+void NO_INLINE Aggregator::executeWithoutKeyImpl(
+ AggregatedDataWithoutKey & res,
+ size_t row_begin, size_t row_end,
+ AggregateFunctionInstruction * aggregate_instructions,
+ Arena * arena) const
+{
+ if (row_begin == row_end)
+ return;
+
+ /// Adding values
+ for (size_t i = 0; i < aggregate_functions.size(); ++i)
+ {
+ AggregateFunctionInstruction * inst = aggregate_instructions + i;
+
+ inst->batch_that->addBatchSinglePlace(
+ row_begin, row_end,
+ res + inst->state_offset,
+ inst->batch_arguments,
+ arena);
+ }
+}
+
+void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns,
+ AggregateFunctionInstructions & aggregate_functions_instructions) const
+{
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_columns[i].resize(params.aggregates[i].arguments.size());
+
+ aggregate_functions_instructions.resize(params.aggregates_size + 1);
+ aggregate_functions_instructions[params.aggregates_size].that = nullptr;
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ {
+ for (size_t j = 0; j < aggregate_columns[i].size(); ++j)
+ {
+ materialized_columns.push_back(columns.at(params.aggregates[i].arguments[j]));
+ aggregate_columns[i][j] = materialized_columns.back().get();
+ }
+
+ aggregate_functions_instructions[i].arguments = aggregate_columns[i].data();
+ aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i];
+
+ const auto * that = aggregate_functions[i];
+
+ aggregate_functions_instructions[i].that = that;
+ aggregate_functions_instructions[i].batch_arguments = aggregate_columns[i].data();
+
+ aggregate_functions_instructions[i].batch_that = that;
+ }
+}
+
+
+bool Aggregator::executeOnBlock(const Block & block,
+ AggregatedDataVariants & result,
+ ColumnRawPtrs & key_columns,
+ AggregateColumns & aggregate_columns,
+ bool & no_more_keys) const
+{
+ return executeOnBlock(block->columns(),
+ /* row_begin= */ 0, block->num_rows(),
+ result,
+ key_columns,
+ aggregate_columns,
+ no_more_keys);
+}
+
+
+bool Aggregator::executeOnBlock(Columns columns,
+ size_t row_begin, size_t row_end,
+ AggregatedDataVariants & result,
+ ColumnRawPtrs & key_columns,
+ AggregateColumns & aggregate_columns,
+ bool & no_more_keys) const
+{
+ /// `result` will destroy the states of aggregate functions in the destructor
+ result.aggregator = this;
+
+ /// How to perform the aggregation?
+ if (result.empty())
+ {
+ result.init(method_chosen);
+ result.keys_size = params.keys_size;
+ result.key_sizes = key_sizes;
+ }
+
+ /** Constant columns are not supported directly during aggregation.
+ * To make them work anyway, we materialize them.
+ */
+ Columns materialized_columns;
+
+ /// Remember the columns we will work with
+ for (size_t i = 0; i < params.keys_size; ++i)
+ {
+ materialized_columns.push_back(columns.at(params.keys[i]));
+ key_columns[i] = materialized_columns.back().get();
+ }
+
+ //NestedColumnsHolder nested_columns_holder;
+ AggregateFunctionInstructions aggregate_functions_instructions;
+ prepareAggregateInstructions(columns, aggregate_columns, materialized_columns, aggregate_functions_instructions);
+
+ if ((params.overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key)
+ {
+ AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
+ createAggregateStates(place);
+ result.without_key = place;
+ }
+
+ /// We select one of the aggregation methods and call it.
+
+ /// For the case when there are no keys (all aggregate into one row).
+ if (result.type == AggregatedDataVariants::Type::without_key)
+ {
+ executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
+ }
+ else
+ {
+ /// This is where data is written that does not fit in `max_rows_to_group_by` with `group_by_overflow_mode = any`.
+ AggregateDataPtr overflow_row_ptr = params.overflow_row ? result.without_key : nullptr;
+
+ #define M(NAME) \
+ else if (result.type == AggregatedDataVariants::Type::NAME) \
+ executeImpl(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_functions_instructions.data(), \
+ no_more_keys, overflow_row_ptr);
+
+ if (false) {} // NOLINT
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ }
+
+ size_t result_size = result.sizeWithoutOverflowRow();
+
+ /// Checking the constraints.
+ if (!checkLimits(result_size, no_more_keys))
+ return false;
+
+ return true;
+}
+
+
+template <typename Method>
+Block Aggregator::convertOneBucketToBlock(
+ AggregatedDataVariants & data_variants,
+ Method & method,
+ Arena * arena,
+ bool final,
+ size_t bucket) const
+{
+ Block block = prepareBlockAndFill(data_variants, final, method.data.impls[bucket].size(),
+ [bucket, &method, arena, this] (
+ MutableColumns & key_columns,
+ AggregateColumnsData & aggregate_columns,
+ MutableColumns & final_aggregate_columns,
+ bool final_)
+ {
+ convertToBlockImpl(method, method.data.impls[bucket],
+ key_columns, aggregate_columns, final_aggregate_columns, arena, final_);
+ });
+
+ //block.info.bucket_num = bucket;
+ return block;
+}
+
+
+bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const
+{
+ if (!no_more_keys && params.max_rows_to_group_by && result_size > params.max_rows_to_group_by)
+ {
+ switch (params.group_by_overflow_mode)
+ {
+ case OverflowMode::THROW:
+ throw Exception("Limit for rows to GROUP BY exceeded");
+
+ case OverflowMode::BREAK:
+ return false;
+
+ case OverflowMode::ANY:
+ no_more_keys = true;
+ break;
+ }
+ }
+
+ return true;
+}
+
+void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVariants & result)
+{
+ ColumnRawPtrs key_columns(params.keys_size);
+ AggregateColumns aggregate_columns(params.aggregates_size);
+
+ /** Used if there is a limit on the maximum number of rows in the aggregation,
+ * and if group_by_overflow_mode == ANY.
+ * In this case, new keys are not added to the set, but aggregation is performed only by
+ * keys that have already managed to get into the set.
+ */
+ bool no_more_keys = false;
+
+ /// Read all the data
+ while (Block block = stream->read())
+ {
+ if (!executeOnBlock(block, result, key_columns, aggregate_columns, no_more_keys))
+ break;
+ }
+
+ /// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation.
+ /// To do this, we pass a block with zero rows to aggregate.
+ if (result.empty() && params.keys_size == 0 && !params.empty_result_for_aggregation_by_empty_set)
+ {
+ auto emptyColumns = columnsFromHeader(stream->getHeader());
+ executeOnBlock(emptyColumns, 0, 0, result, key_columns, aggregate_columns, no_more_keys);
+ }
+}
+
+
+template <typename Method, typename Table>
+void Aggregator::convertToBlockImpl(
+ Method & method,
+ Table & data,
+ MutableColumns & key_columns,
+ AggregateColumnsData & aggregate_columns,
+ MutableColumns & final_aggregate_columns,
+ Arena * arena,
+ bool final) const
+{
+ if (data.empty())
+ return;
+
+ if (key_columns.size() != params.keys_size)
+ throw Exception{"Aggregate. Unexpected key columns size."};
+
+ if (final)
+ convertToBlockImplFinal<Method>(method, data, key_columns, final_aggregate_columns, arena);
+ else
+ convertToBlockImplNotFinal(method, data, key_columns, aggregate_columns);
+
+ /// In order to release memory early.
+ data.clearAndShrink();
+}
+
+
+template <typename Mapped>
+inline void Aggregator::insertAggregatesIntoColumns(
+ Mapped & mapped,
+ MutableColumns & final_aggregate_columns,
+ Arena * arena) const
+{
+ /** Final values of aggregate functions are inserted to columns.
+ * Then states of aggregate functions, that are not longer needed, are destroyed.
+ *
+ * We mark already destroyed states with "nullptr" in data,
+ * so they will not be destroyed in destructor of Aggregator
+ * (other values will be destroyed in destructor in case of exception).
+ *
+ * But it becomes tricky, because we have multiple aggregate states pointed by a single pointer in data.
+ * So, if exception is thrown in the middle of moving states for different aggregate functions,
+ * we have to catch exceptions and destroy all the states that are no longer needed,
+ * to keep the data in consistent state.
+ *
+ * It is also tricky, because there are aggregate functions with "-State" modifier.
+ * When we call "insertResultInto" for them, they insert a pointer to the state to ColumnAggregateFunction
+ * and ColumnAggregateFunction will take ownership of this state.
+ * So, for aggregate functions with "-State" modifier, the state must not be destroyed
+ * after it has been transferred to ColumnAggregateFunction.
+ * But we should mark that the data no longer owns these states.
+ */
+
+ size_t insert_i = 0;
+ std::exception_ptr exception;
+
+ try
+ {
+ /// Insert final values of aggregate functions into columns.
+ for (; insert_i < params.aggregates_size; ++insert_i)
+ aggregate_functions[insert_i]->insertResultInto(
+ mapped + offsets_of_aggregate_states[insert_i],
+ *final_aggregate_columns[insert_i],
+ arena);
+ }
+ catch (...)
+ {
+ exception = std::current_exception();
+ }
+
+ /** Destroy states that are no longer needed. This loop does not throw.
+ *
+ * Don't destroy states for "-State" aggregate functions,
+ * because the ownership of this state is transferred to ColumnAggregateFunction
+ * and ColumnAggregateFunction will take care.
+ *
+ * But it's only for states that has been transferred to ColumnAggregateFunction
+ * before exception has been thrown;
+ */
+ for (size_t destroy_i = 0; destroy_i < params.aggregates_size; ++destroy_i)
+ {
+ /// If ownership was not transferred to ColumnAggregateFunction.
+ if (!(destroy_i < insert_i && aggregate_functions[destroy_i]->isState()))
+ aggregate_functions[destroy_i]->destroy(
+ mapped + offsets_of_aggregate_states[destroy_i]);
+ }
+
+ /// Mark the cell as destroyed so it will not be destroyed in destructor.
+ mapped = nullptr;
+
+ if (exception)
+ std::rethrow_exception(exception);
+}
+
+
+template <typename Method, typename Table>
+void NO_INLINE Aggregator::convertToBlockImplFinal(
+ Method & method,
+ Table & data,
+ const MutableColumns & key_columns,
+ MutableColumns & final_aggregate_columns,
+ Arena * arena) const
+{
+#if 0 // TODO: enable shuffle in AggregationMethodKeysFixed
+ std::vector<MutableColumn *> raw_key_columns;
+ raw_key_columns.reserve(key_columns.size());
+ for (auto & column : key_columns)
+ raw_key_columns.push_back(column.get());
+
+ //auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes);
+ //const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes;
+#endif
+ const auto & key_sizes_ref = key_sizes;
+
+ PaddedPODArray<AggregateDataPtr> places;
+ places.reserve(data.size());
+
+ data.forEachValue([&](const auto & key, auto & mapped)
+ {
+ method.insertKeyIntoColumns(key, key_columns, key_sizes_ref);
+ places.emplace_back(mapped);
+
+ /// Mark the cell as destroyed so it will not be destroyed in destructor.
+ mapped = nullptr;
+ });
+
+ std::exception_ptr exception;
+ size_t aggregate_functions_destroy_index = 0;
+
+ try
+ {
+ for (; aggregate_functions_destroy_index < params.aggregates_size;)
+ {
+ auto & final_aggregate_column = final_aggregate_columns[aggregate_functions_destroy_index];
+ size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index];
+
+ /** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch
+ * throws exception, it also must destroy all necessary states.
+ * Then code need to continue to destroy other aggregate function states with next function index.
+ */
+ size_t destroy_index = aggregate_functions_destroy_index;
+ ++aggregate_functions_destroy_index;
+
+ /// For State AggregateFunction ownership of aggregate place is passed to result column after insert
+ bool is_state = aggregate_functions[destroy_index]->isState();
+ bool destroy_place_after_insert = !is_state;
+
+ aggregate_functions[destroy_index]->insertResultIntoBatch(0, places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert);
+ }
+ }
+ catch (...)
+ {
+ exception = std::current_exception();
+ }
+
+ for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index)
+ {
+ size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index];
+ aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset);
+ }
+
+ if (exception)
+ std::rethrow_exception(exception);
+}
+
+template <typename Method, typename Table>
+void NO_INLINE Aggregator::convertToBlockImplNotFinal(
+ Method & method,
+ Table & data,
+ const MutableColumns & key_columns,
+ AggregateColumnsData & aggregate_columns) const
+{
+#if 0 // TODO: enable shuffle in AggregationMethodKeysFixed
+ std::vector<MutableColumn *> raw_key_columns;
+ raw_key_columns.reserve(key_columns.size());
+ for (auto & column : key_columns)
+ raw_key_columns.push_back(column.get());
+
+ //auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes);
+ //const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes;
+#endif
+ const auto & key_sizes_ref = key_sizes;
+
+ data.forEachValue([&](const auto & key, auto & mapped)
+ {
+ method.insertKeyIntoColumns(key, key_columns, key_sizes_ref);
+
+ /// reserved, so push_back does not throw exceptions
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_columns[i]->Append(reinterpret_cast<uint64_t>(mapped + offsets_of_aggregate_states[i])).ok();
+
+ mapped = nullptr;
+ });
+}
+
+
+template <typename Filler>
+Block Aggregator::prepareBlockAndFill(
+ AggregatedDataVariants & /*data_variants*/,
+ bool final,
+ size_t rows,
+ Filler && filler) const
+{
+ Header header = getHeader(final);
+
+ std::vector<std::shared_ptr<MutableColumnAggregateFunction>> aggregate_columns(params.aggregates_size);
+ MutableColumns final_aggregate_columns(params.aggregates_size);
+ AggregateColumnsData aggregate_columns_data(params.aggregates_size);
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ {
+ if (!final)
+ {
+ const auto & aggregate_column_name = params.aggregates[i].column_name;
+ auto & type = header->GetFieldByName(aggregate_column_name)->type();
+ aggregate_columns[i] = std::make_shared<MutableColumnAggregateFunction>(
+ std::static_pointer_cast<DataTypeAggregateFunction>(type)); // TODO: set pool
+
+ /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states.
+ auto & column_aggregate_func = *aggregate_columns[i];
+#if 0
+ for (auto & pool : data_variants.aggregates_pools)
+ column_aggregate_func.addArena(pool);
+#endif
+ aggregate_columns_data[i] = &column_aggregate_func.getData();
+ aggregate_columns_data[i]->Reserve(rows).ok();
+ }
+ else
+ {
+ final_aggregate_columns[i] = createMutableColumn(aggregate_functions[i]->getReturnType());
+ final_aggregate_columns[i]->Reserve(rows).ok();
+ }
+ }
+
+ MutableColumns key_columns(params.keys_size);
+ for (size_t i = 0; i < params.keys_size; ++i)
+ {
+ key_columns[i] = createMutableColumn(header->field(i)->type());
+ key_columns[i]->Reserve(rows).ok();
+ }
+
+ filler(key_columns, aggregate_columns_data, final_aggregate_columns, final);
+
+ Columns columns(params.keys_size + params.aggregates_size);
+
+ for (size_t i = 0; i < params.keys_size; ++i)
+ columns[i] = *key_columns[i]->Finish();
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ {
+ int pos = header->GetFieldIndex(params.aggregates[i].column_name);
+ if (final)
+ columns[pos] = *final_aggregate_columns[i]->Finish();
+ else
+ columns[pos] = *aggregate_columns[i]->Finish();
+ }
+
+ // TODO: check row == columns length()
+ return arrow::RecordBatch::Make(header, rows, columns);
+}
+
+Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool /*is_overflows*/) const
+{
+ size_t rows = 1;
+
+ auto filler = [&data_variants, this](
+ MutableColumns & key_columns,
+ AggregateColumnsData & aggregate_columns,
+ MutableColumns & final_aggregate_columns,
+ bool final_)
+ {
+ if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
+ {
+ AggregatedDataWithoutKey & data = data_variants.without_key;
+
+ if (!data)
+ throw Exception("Wrong data variant passed.");
+
+ if (!final_)
+ {
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_columns[i]->Append(reinterpret_cast<uint64_t>(data + offsets_of_aggregate_states[i])).ok();
+ data = nullptr;
+ }
+ else
+ {
+ /// Always single-thread. It's safe to pass current arena from 'aggregates_pool'.
+ insertAggregatesIntoColumns(data, final_aggregate_columns, data_variants.aggregates_pool);
+ }
+
+ if (params.overflow_row)
+ for (size_t i = 0; i < params.keys_size; ++i)
+ key_columns[i]->AppendEmptyValue().ok(); // FIXME: or AppendNull() ???
+ }
+ };
+
+ Block block = prepareBlockAndFill(data_variants, final, rows, filler);
+#if 0
+ if (is_overflows)
+ block.info.is_overflows = true;
+#endif
+ if (final)
+ destroyWithoutKey(data_variants);
+
+ return block;
+}
+
+Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const
+{
+ size_t rows = data_variants.sizeWithoutOverflowRow();
+
+ auto filler = [&data_variants, this](
+ MutableColumns & key_columns,
+ AggregateColumnsData & aggregate_columns,
+ MutableColumns & final_aggregate_columns,
+ bool final_)
+ {
+ #define M(NAME) \
+ else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
+ convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, \
+ key_columns, aggregate_columns, final_aggregate_columns, data_variants.aggregates_pool, final_);
+
+ if (false) {} // NOLINT
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ else
+ throw Exception("Unknown aggregated data variant.");
+ };
+
+ return prepareBlockAndFill(data_variants, final, rows, filler);
+}
+
+
+BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, bool final) const
+{
+ BlocksList blocks;
+
+ /// In what data structure is the data aggregated?
+ if (data_variants.empty())
+ return blocks;
+
+ if (data_variants.without_key)
+ blocks.emplace_back(prepareBlockAndFillWithoutKey(
+ data_variants, final, data_variants.type != AggregatedDataVariants::Type::without_key));
+
+ if (data_variants.type != AggregatedDataVariants::Type::without_key)
+ {
+ blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final));
+ }
+
+ if (!final)
+ {
+ /// data_variants will not destroy the states of aggregate functions in the destructor.
+ /// Now ColumnAggregateFunction owns the states.
+ data_variants.aggregator = nullptr;
+ }
+
+ size_t rows = 0;
+ //size_t bytes = 0;
+
+ for (const auto & block : blocks)
+ {
+ rows += block->num_rows();
+ //bytes += block.bytes();
+ }
+
+ return blocks;
+}
+
+
+template <typename Method, typename Table>
+void NO_INLINE Aggregator::mergeDataImpl(
+ Table & table_dst,
+ Table & table_src,
+ Arena * arena) const
+{
+ table_src.mergeToViaEmplace(table_dst, [&](AggregateDataPtr & __restrict dst, AggregateDataPtr & __restrict src, bool inserted)
+ {
+ if (!inserted)
+ {
+ {
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->merge(dst + offsets_of_aggregate_states[i], src + offsets_of_aggregate_states[i], arena);
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->destroy(src + offsets_of_aggregate_states[i]);
+ }
+ }
+ else
+ {
+ dst = src;
+ }
+
+ src = nullptr;
+ });
+
+ table_src.clearAndShrink();
+}
+
+
+template <typename Method, typename Table>
+void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl(
+ Table & table_dst,
+ AggregatedDataWithoutKey & overflows,
+ Table & table_src,
+ Arena * arena) const
+{
+ table_src.mergeToViaFind(table_dst, [&](AggregateDataPtr dst, AggregateDataPtr & src, bool found)
+ {
+ AggregateDataPtr res_data = found ? dst : overflows;
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->merge(
+ res_data + offsets_of_aggregate_states[i],
+ src + offsets_of_aggregate_states[i],
+ arena);
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->destroy(src + offsets_of_aggregate_states[i]);
+
+ src = nullptr;
+ });
+ table_src.clearAndShrink();
+}
+
+template <typename Method, typename Table>
+void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl(
+ Table & table_dst,
+ Table & table_src,
+ Arena * arena) const
+{
+ table_src.mergeToViaFind(table_dst,
+ [&](AggregateDataPtr dst, AggregateDataPtr & src, bool found)
+ {
+ if (!found)
+ return;
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->merge(
+ dst + offsets_of_aggregate_states[i],
+ src + offsets_of_aggregate_states[i],
+ arena);
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->destroy(src + offsets_of_aggregate_states[i]);
+
+ src = nullptr;
+ });
+ table_src.clearAndShrink();
+}
+
+
+void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(
+ ManyAggregatedDataVariants & non_empty_data) const
+{
+ AggregatedDataVariantsPtr & res = non_empty_data[0];
+
+ /// We merge all aggregation results to the first.
+ for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num)
+ {
+ AggregatedDataWithoutKey & res_data = res->without_key;
+ AggregatedDataWithoutKey & current_data = non_empty_data[result_num]->without_key;
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->merge(res_data + offsets_of_aggregate_states[i], current_data + offsets_of_aggregate_states[i], res->aggregates_pool);
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->destroy(current_data + offsets_of_aggregate_states[i]);
+
+ current_data = nullptr;
+ }
+}
+
+
+template <typename Method>
+void NO_INLINE Aggregator::mergeSingleLevelDataImpl(
+ ManyAggregatedDataVariants & non_empty_data) const
+{
+ AggregatedDataVariantsPtr & res = non_empty_data[0];
+ bool no_more_keys = false;
+
+ /// We merge all aggregation results to the first.
+ for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num)
+ {
+ if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
+ break;
+
+ AggregatedDataVariants & current = *non_empty_data[result_num];
+
+ if (!no_more_keys)
+ {
+ {
+ mergeDataImpl<Method>(
+ getDataVariant<Method>(*res).data,
+ getDataVariant<Method>(current).data,
+ res->aggregates_pool);
+ }
+ }
+ else if (res->without_key)
+ {
+ mergeDataNoMoreKeysImpl<Method>(
+ getDataVariant<Method>(*res).data,
+ res->without_key,
+ getDataVariant<Method>(current).data,
+ res->aggregates_pool);
+ }
+ else
+ {
+ mergeDataOnlyExistingKeysImpl<Method>(
+ getDataVariant<Method>(*res).data,
+ getDataVariant<Method>(current).data,
+ res->aggregates_pool);
+ }
+
+ /// `current` will not destroy the states of aggregate functions in the destructor
+ current.aggregator = nullptr;
+ }
+}
+
+#define M(NAME) \
+ template void NO_INLINE Aggregator::mergeSingleLevelDataImpl<decltype(AggregatedDataVariants::NAME)::element_type>( \
+ ManyAggregatedDataVariants & non_empty_data) const;
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+#undef M
+
+template <typename Method>
+void NO_INLINE Aggregator::mergeBucketImpl(
+ ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena, std::atomic<bool> * is_cancelled) const
+{
+ /// We merge all aggregation results to the first.
+ AggregatedDataVariantsPtr & res = data[0];
+ for (size_t result_num = 1, size = data.size(); result_num < size; ++result_num)
+ {
+ if (is_cancelled && is_cancelled->load(std::memory_order_seq_cst))
+ return;
+
+ AggregatedDataVariants & current = *data[result_num];
+
+ {
+ mergeDataImpl<Method>(
+ getDataVariant<Method>(*res).data.impls[bucket],
+ getDataVariant<Method>(current).data.impls[bucket],
+ arena);
+ }
+ }
+}
+
+ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedDataVariants & data_variants) const
+{
+ if (data_variants.empty())
+ throw Exception("Empty data passed to Aggregator::mergeAndConvertToBlocks.");
+
+ ManyAggregatedDataVariants non_empty_data;
+ non_empty_data.reserve(data_variants.size());
+ for (auto & data : data_variants)
+ if (!data->empty())
+ non_empty_data.push_back(data);
+
+ if (non_empty_data.empty())
+ return {};
+
+ if (non_empty_data.size() > 1)
+ {
+ /// Sort the states in descending order so that the merge is more efficient (since all states are merged into the first).
+ std::sort(non_empty_data.begin(), non_empty_data.end(),
+ [](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs)
+ {
+ return lhs->sizeWithoutOverflowRow() > rhs->sizeWithoutOverflowRow();
+ });
+ }
+
+ AggregatedDataVariantsPtr & first = non_empty_data[0];
+
+ for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
+ {
+ if (first->type != non_empty_data[i]->type)
+ throw Exception("Cannot merge different aggregated data variants.");
+
+ /** Elements from the remaining sets can be moved to the first data set.
+ * Therefore, it must own all the arenas of all other sets.
+ */
+ first->aggregates_pools.insert(first->aggregates_pools.end(),
+ non_empty_data[i]->aggregates_pools.begin(), non_empty_data[i]->aggregates_pools.end());
+ }
+
+ return non_empty_data;
+}
+
+template <bool no_more_keys, typename Method, typename Table>
+void NO_INLINE Aggregator::mergeStreamsImplCase(
+ Block & block,
+ Arena * aggregates_pool,
+ Method & method [[maybe_unused]],
+ Table & data,
+ AggregateDataPtr overflow_row) const
+{
+ ColumnRawPtrs key_columns(params.keys_size);
+ std::vector<const ColumnAggregateFunction *> aggregate_columns(params.aggregates_size);
+
+ /// Remember the columns we will work with
+ for (size_t i = 0; i < params.keys_size; ++i)
+ key_columns[i] = block->column(i).get();
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ {
+ const auto & aggregate_column_name = params.aggregates[i].column_name;
+ aggregate_columns[i] = &assert_cast<const ColumnAggregateFunction &>(*block->GetColumnByName(aggregate_column_name));
+ }
+
+ typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
+
+ /// For all rows.
+ size_t rows = block->num_rows();
+ std::unique_ptr<AggregateDataPtr[]> places(new AggregateDataPtr[rows]);
+
+ for (size_t i = 0; i < rows; ++i)
+ {
+ AggregateDataPtr aggregate_data = nullptr;
+
+ if (!no_more_keys)
+ {
+ auto emplace_result = state.emplaceKey(data, i, *aggregates_pool);
+ if (emplace_result.isInserted())
+ {
+ emplace_result.setMapped(nullptr);
+
+ aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
+ createAggregateStates(aggregate_data);
+
+ emplace_result.setMapped(aggregate_data);
+ }
+ else
+ aggregate_data = emplace_result.getMapped();
+ }
+ else
+ {
+ auto find_result = state.findKey(data, i, *aggregates_pool);
+ if (find_result.isFound())
+ aggregate_data = find_result.getMapped();
+ }
+
+ /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys.
+
+ AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row;
+ places[i] = value;
+ }
+
+ for (size_t j = 0; j < params.aggregates_size; ++j)
+ {
+ /// Merge state of aggregate functions.
+ aggregate_functions[j]->mergeBatch(
+ 0, rows,
+ places.get(), offsets_of_aggregate_states[j],
+ aggregate_columns[j]->rawData(),
+ aggregates_pool);
+ }
+
+ /// Early release memory.
+ block.reset();
+}
+
+template <typename Method, typename Table>
+void NO_INLINE Aggregator::mergeStreamsImpl(
+ Block & block,
+ Arena * aggregates_pool,
+ Method & method,
+ Table & data,
+ AggregateDataPtr overflow_row,
+ bool no_more_keys) const
+{
+ if (!no_more_keys)
+ mergeStreamsImplCase<false>(block, aggregates_pool, method, data, overflow_row);
+ else
+ mergeStreamsImplCase<true>(block, aggregates_pool, method, data, overflow_row);
+}
+
+
+void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
+ Block & block,
+ AggregatedDataVariants & result) const
+{
+ std::vector<const ColumnAggregateFunction *> aggregate_columns(params.aggregates_size);
+
+ /// Remember the columns we will work with
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ {
+ const auto & aggregate_column_name = params.aggregates[i].column_name;
+ aggregate_columns[i] = &assert_cast<const ColumnAggregateFunction &>(*block->GetColumnByName(aggregate_column_name));
+ }
+
+ AggregatedDataWithoutKey & res = result.without_key;
+ if (!res)
+ {
+ AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
+ createAggregateStates(place);
+ res = place;
+ }
+
+ for (size_t row = 0, rows = block->num_rows(); row < rows; ++row)
+ {
+ /// Adding Values
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->merge(res + offsets_of_aggregate_states[i],
+ aggregate_columns[i]->rawData()[row], result.aggregates_pool);
+ }
+
+ /// Early release memory.
+ block.reset();
+}
+
+bool Aggregator::mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const
+{
+ /// `result` will destroy the states of aggregate functions in the destructor
+ result.aggregator = this;
+
+ /// How to perform the aggregation?
+ if (result.empty())
+ {
+ result.init(method_chosen);
+ result.keys_size = params.keys_size;
+ result.key_sizes = key_sizes;
+ }
+
+ if (result.type == AggregatedDataVariants::Type::without_key /*|| block.info.is_overflows*/)
+ mergeWithoutKeyStreamsImpl(block, result);
+
+#define M(NAME) \
+ else if (result.type == AggregatedDataVariants::Type::NAME) \
+ mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
+
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+#undef M
+ else if (result.type != AggregatedDataVariants::Type::without_key)
+ throw Exception("Unknown aggregated data variant.");
+
+ size_t result_size = result.sizeWithoutOverflowRow();
+
+ /// Checking the constraints.
+ if (!checkLimits(result_size, no_more_keys))
+ return false;
+
+ return true;
+}
+
+void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataVariants & result)
+{
+#if 0
+ if (isCancelled())
+ return;
+
+ /** If the remote servers used a two-level aggregation method,
+ * then blocks will contain information about the number of the bucket.
+ * Then the calculations can be parallelized by buckets.
+ * We decompose the blocks to the bucket numbers indicated in them.
+ */
+ BucketToBlocks bucket_to_blocks;
+
+ while (Block block = stream->read())
+ {
+ if (isCancelled())
+ return;
+
+ bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block));
+ }
+
+ mergeBlocks(bucket_to_blocks, result);
+#else
+ BlocksList blocks;
+
+ while (Block block = stream->read())
+ blocks.emplace_back(std::move(block));
+
+ BucketToBlocks bucket_to_blocks;
+ bucket_to_blocks.emplace(-1, std::move(blocks));
+ mergeBlocks(std::move(bucket_to_blocks), result);
+#endif
+}
+
+void Aggregator::mergeBlocks(BucketToBlocks && bucket_to_blocks, AggregatedDataVariants & result)
+{
+ if (bucket_to_blocks.empty())
+ return;
+
+ /** `minus one` means the absence of information about the bucket
+ * - in the case of single-level aggregation, as well as for blocks with "overflowing" values.
+ * If there is at least one block with a bucket number greater or equal than zero, then there was a two-level aggregation.
+ */
+ //auto max_bucket = bucket_to_blocks.rbegin()->first;
+
+ /// result will destroy the states of aggregate functions in the destructor
+ result.aggregator = this;
+
+ result.init(method_chosen);
+ result.keys_size = params.keys_size;
+ result.key_sizes = key_sizes;
+
+ bool has_blocks_with_unknown_bucket = bucket_to_blocks.count(-1);
+
+ if (has_blocks_with_unknown_bucket)
+ {
+ bool no_more_keys = false;
+
+ BlocksList & blocks = bucket_to_blocks[-1];
+ for (Block & block : blocks)
+ {
+ if (!checkLimits(result.sizeWithoutOverflowRow(), no_more_keys))
+ break;
+
+ if (result.type == AggregatedDataVariants::Type::without_key /*|| block.info.is_overflows*/)
+ mergeWithoutKeyStreamsImpl(block, result);
+
+ #define M(NAME) \
+ else if (result.type == AggregatedDataVariants::Type::NAME) \
+ mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
+
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ else if (result.type != AggregatedDataVariants::Type::without_key)
+ throw Exception("Unknown aggregated data variant.");
+ }
+ }
+}
+
+
+Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
+{
+ if (blocks.empty())
+ return {};
+
+#if 0
+ auto bucket_num = blocks.front().info.bucket_num;
+ bool is_overflows = blocks.front().info.is_overflows;
+#endif
+
+ /** If possible, change 'method' to some_hash64. Otherwise, leave as is.
+ * Better hash function is needed because during external aggregation,
+ * we may merge partitions of data with total number of keys far greater than 4 billion.
+ */
+ auto merge_method = method_chosen;
+
+#define APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION(M) \
+ M(key64) \
+ M(key_string) \
+ M(key_fixed_string) \
+ M(keys128) \
+ M(keys256) \
+ M(serialized) \
+
+#define M(NAME) \
+ if (merge_method == AggregatedDataVariants::Type::NAME) \
+ merge_method = AggregatedDataVariants::Type::NAME ## _hash64; \
+
+ APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION(M)
+#undef M
+
+#undef APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION
+
+ /// Temporary data for aggregation.
+ AggregatedDataVariants result;
+
+ /// result will destroy the states of aggregate functions in the destructor
+ result.aggregator = this;
+
+ result.init(merge_method);
+ result.keys_size = params.keys_size;
+ result.key_sizes = key_sizes;
+
+ for (Block & block : blocks)
+ {
+#if 0
+ if (bucket_num >= 0 /*&& block.info.bucket_num != bucket_num*/)
+ bucket_num = -1;
+#endif
+ if (result.type == AggregatedDataVariants::Type::without_key /*|| is_overflows*/)
+ mergeWithoutKeyStreamsImpl(block, result);
+
+ #define M(NAME) \
+ else if (result.type == AggregatedDataVariants::Type::NAME) \
+ mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false);
+
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ else if (result.type != AggregatedDataVariants::Type::without_key)
+ throw Exception("Unknown aggregated data variant.");
+ }
+
+ Block block;
+ if (result.type == AggregatedDataVariants::Type::without_key /*|| is_overflows*/)
+ block = prepareBlockAndFillWithoutKey(result, final /*, is_overflows*/);
+ else
+ block = prepareBlockAndFillSingleLevel(result, final);
+ /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods.
+
+ if (!final)
+ {
+ /// Pass ownership of aggregate function states from result to ColumnAggregateFunction objects in the resulting block.
+ result.aggregator = nullptr;
+ }
+
+ //block.info.bucket_num = bucket_num;
+ return block;
+}
+
+
+template <typename Method, typename Table>
+void NO_INLINE Aggregator::destroyImpl(Table & table) const
+{
+ table.forEachMapped([&](AggregateDataPtr & data)
+ {
+ /** If an exception (usually a lack of memory, the MemoryTracker throws) arose
+ * after inserting the key into a hash table, but before creating all states of aggregate functions,
+ * then data will be equal nullptr.
+ */
+ if (nullptr == data)
+ return;
+
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->destroy(data + offsets_of_aggregate_states[i]);
+
+ data = nullptr;
+ });
+}
+
+
+void Aggregator::destroyWithoutKey(AggregatedDataVariants & result) const
+{
+ AggregatedDataWithoutKey & res_data = result.without_key;
+
+ if (nullptr != res_data)
+ {
+ for (size_t i = 0; i < params.aggregates_size; ++i)
+ aggregate_functions[i]->destroy(res_data + offsets_of_aggregate_states[i]);
+
+ res_data = nullptr;
+ }
+}
+
+
+void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) const
+{
+ if (result.empty())
+ return;
+
+ /// In what data structure is the data aggregated?
+ if (result.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
+ destroyWithoutKey(result);
+
+#define M(NAME) \
+ else if (result.type == AggregatedDataVariants::Type::NAME) \
+ destroyImpl<decltype(result.NAME)::element_type>(result.NAME->data);
+
+ if (false) {} // NOLINT
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+#undef M
+ else if (result.type != AggregatedDataVariants::Type::without_key)
+ throw Exception("Unknown aggregated data variant.");
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/Aggregator.h b/ydb/library/arrow_clickhouse/Aggregator.h
new file mode 100644
index 00000000000..a83baf531f1
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Aggregator.h
@@ -0,0 +1,923 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <memory>
+#include <common/StringRef.h>
+
+#include "AggregationCommon.h"
+#include <Common/Arena.h>
+#include <Common/HashTable/FixedHashMap.h>
+#include <Common/HashTable/HashMap.h>
+#include <Common/HashTable/StringHashMap.h>
+#include <Columns/ColumnsHashing.h>
+#include <Columns/ColumnAggregateFunction.h>
+#include <DataStreams/IBlockStream_fwd.h>
+
+namespace CH
+{
+
+struct AggregateDescription
+{
+ AggregateFunctionPtr function;
+ Array parameters; /// Parameters of the (parametric) aggregate function.
+ ColumnNumbers arguments;
+ Names argument_names; /// used if no `arguments` are specified.
+ String column_name; /// What name to use for a column with aggregate function values
+};
+
+using AggregateDescriptions = std::vector<AggregateDescription>;
+
+/** Different data structures that can be used for aggregation
+ * For efficiency, the aggregation data itself is put into the pool.
+ * Data and pool ownership (states of aggregate functions)
+ * is acquired later - in `convertToBlocks` function, by the ColumnAggregateFunction object.
+ *
+ * Most data structures exist in two versions: normal and two-level (TwoLevel).
+ * A two-level hash table works a little slower with a small number of different keys,
+ * but with a large number of different keys scales better, because it allows
+ * parallelize some operations (merging, post-processing) in a natural way.
+ *
+ * To ensure efficient work over a wide range of conditions,
+ * first single-level hash tables are used,
+ * and when the number of different keys is large enough,
+ * they are converted to two-level ones.
+ *
+ * PS. There are many different approaches to the effective implementation of parallel and distributed aggregation,
+ * best suited for different cases, and this approach is just one of them, chosen for a combination of reasons.
+ */
+
+using AggregateDataPtr = char *;
+using AggregatedDataWithoutKey = AggregateDataPtr;
+
+using AggregatedDataWithUInt8Key = FixedImplicitZeroHashMapWithCalculatedSize<UInt8, AggregateDataPtr>;
+using AggregatedDataWithUInt16Key = FixedImplicitZeroHashMap<UInt16, AggregateDataPtr>;
+
+using AggregatedDataWithUInt32Key = HashMap<UInt32, AggregateDataPtr, HashCRC32<UInt32>>;
+using AggregatedDataWithUInt64Key = HashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>>;
+
+using AggregatedDataWithShortStringKey = StringHashMap<AggregateDataPtr>;
+
+using AggregatedDataWithStringKey = HashMapWithSavedHash<StringRef, AggregateDataPtr>;
+
+using AggregatedDataWithKeys128 = HashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
+using AggregatedDataWithKeys256 = HashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
+
+
+/** Variants with better hash function, using more than 32 bits for hash.
+ * Using for merging phase of external aggregation, where number of keys may be far greater than 4 billion,
+ * but we keep in memory and merge only sub-partition of them simultaneously.
+ * TODO We need to switch for better hash function not only for external aggregation,
+ * but also for huge aggregation results on machines with terabytes of RAM.
+ */
+
+using AggregatedDataWithUInt64KeyHash64 = HashMap<UInt64, AggregateDataPtr, DefaultHash<UInt64>>;
+using AggregatedDataWithStringKeyHash64 = HashMapWithSavedHash<StringRef, AggregateDataPtr, StringRefHash>;
+using AggregatedDataWithKeys128Hash64 = HashMap<UInt128, AggregateDataPtr, UInt128Hash>;
+using AggregatedDataWithKeys256Hash64 = HashMap<UInt256, AggregateDataPtr, UInt256Hash>;
+
+template <typename Base>
+struct AggregationDataWithNullKey : public Base
+{
+ using Base::Base;
+
+ bool & hasNullKeyData() { return has_null_key_data; }
+ AggregateDataPtr & getNullKeyData() { return null_key_data; }
+ bool hasNullKeyData() const { return has_null_key_data; }
+ const AggregateDataPtr & getNullKeyData() const { return null_key_data; }
+ size_t size() const { return Base::size() + (has_null_key_data ? 1 : 0); }
+ bool empty() const { return Base::empty() && !has_null_key_data; }
+ void clear()
+ {
+ Base::clear();
+ has_null_key_data = false;
+ }
+ void clearAndShrink()
+ {
+ Base::clearAndShrink();
+ has_null_key_data = false;
+ }
+
+private:
+ bool has_null_key_data = false;
+ AggregateDataPtr null_key_data = nullptr;
+};
+
+template <typename ... Types>
+using HashTableWithNullKey = AggregationDataWithNullKey<HashMapTable<Types ...>>;
+template <typename ... Types>
+using StringHashTableWithNullKey = AggregationDataWithNullKey<StringHashMap<Types ...>>;
+
+using AggregatedDataWithNullableUInt8Key = AggregationDataWithNullKey<AggregatedDataWithUInt8Key>;
+using AggregatedDataWithNullableUInt16Key = AggregationDataWithNullKey<AggregatedDataWithUInt16Key>;
+
+using AggregatedDataWithNullableUInt64Key = AggregationDataWithNullKey<AggregatedDataWithUInt64Key>;
+using AggregatedDataWithNullableStringKey = AggregationDataWithNullKey<AggregatedDataWithStringKey>;
+
+
+/// For the case where there is one numeric key.
+/// FieldType is UInt8/16/32/64 for any type with corresponding bit width.
+template <typename FieldType, typename TData,
+ bool consecutive_keys_optimization = true>
+struct AggregationMethodOneNumber
+{
+ using Data = TData;
+ using Key = typename Data::key_type;
+ using Mapped = typename Data::mapped_type;
+
+ Data data;
+
+ AggregationMethodOneNumber() = default;
+
+ template <typename Other>
+ AggregationMethodOneNumber(const Other & other) : data(other.data) {}
+
+ /// To use one `Method` in different threads, use different `State`.
+ using State = ColumnsHashing::HashMethodOneNumber<typename Data::value_type,
+ Mapped, FieldType, consecutive_keys_optimization>;
+
+ /// Shuffle key columns before `insertKeyIntoColumns` call if needed.
+ std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
+
+ // Insert the key from the hash table into columns.
+ template <typename ColPtr>
+ static void insertKeyIntoColumns(const Key & key, const std::vector<ColPtr> & key_columns, const Sizes & /*key_sizes*/)
+ {
+ insertNumber(*key_columns[0], key);
+ }
+};
+
+
+/// For the case where there is one string key.
+template <typename TData>
+struct AggregationMethodString
+{
+ using Data = TData;
+ using Key = typename Data::key_type;
+ using Mapped = typename Data::mapped_type;
+
+ Data data;
+
+ AggregationMethodString() = default;
+
+ template <typename Other>
+ AggregationMethodString(const Other & other) : data(other.data) {}
+
+ using State = ColumnsHashing::HashMethodString<typename Data::value_type, Mapped>;
+
+ std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
+
+ template <typename ColPtr>
+ static void insertKeyIntoColumns(const StringRef & key, const std::vector<ColPtr> & key_columns, const Sizes &)
+ {
+ insertString(*key_columns[0], key);
+ }
+};
+
+
+/// Same as above but without cache
+template <typename TData>
+struct AggregationMethodStringNoCache
+{
+ using Data = TData;
+ using Key = typename Data::key_type;
+ using Mapped = typename Data::mapped_type;
+
+ Data data;
+
+ AggregationMethodStringNoCache() = default;
+
+ template <typename Other>
+ AggregationMethodStringNoCache(const Other & other) : data(other.data) {}
+
+ using State = ColumnsHashing::HashMethodString<typename Data::value_type, Mapped, true, false>;
+
+ std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
+
+ template <typename ColPtr>
+ static void insertKeyIntoColumns(const StringRef & key, const std::vector<ColPtr> & key_columns, const Sizes &)
+ {
+ insertString(*key_columns[0], key);
+ }
+};
+
+
+/// For the case where there is one fixed-length string key.
+template <typename TData>
+struct AggregationMethodFixedString
+{
+ using Data = TData;
+ using Key = typename Data::key_type;
+ using Mapped = typename Data::mapped_type;
+
+ Data data;
+
+ AggregationMethodFixedString() = default;
+
+ template <typename Other>
+ AggregationMethodFixedString(const Other & other) : data(other.data) {}
+
+ using State = ColumnsHashing::HashMethodFixedString<typename Data::value_type, Mapped>;
+
+ std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
+
+ template <typename ColPtr>
+ static void insertKeyIntoColumns(const StringRef & key, const std::vector<ColPtr> & key_columns, const Sizes &)
+ {
+ insertFixedString(*key_columns[0], key);
+ }
+};
+
+/// Same as above but without cache
+template <typename TData>
+struct AggregationMethodFixedStringNoCache
+{
+ using Data = TData;
+ using Key = typename Data::key_type;
+ using Mapped = typename Data::mapped_type;
+
+ Data data;
+
+ AggregationMethodFixedStringNoCache() = default;
+
+ template <typename Other>
+ AggregationMethodFixedStringNoCache(const Other & other) : data(other.data) {}
+
+ using State = ColumnsHashing::HashMethodFixedString<typename Data::value_type, Mapped, true, false>;
+
+ std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
+
+ template <typename ColPtr>
+ static void insertKeyIntoColumns(const StringRef & key, const std::vector<ColPtr> & key_columns, const Sizes &)
+ {
+ insertFixedString(*key_columns[0], key);
+ }
+};
+
+
+
+/// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits.
+template <typename TData, bool has_nullable_keys_ = false, bool use_cache = true>
+struct AggregationMethodKeysFixed
+{
+ using Data = TData;
+ using Key = typename Data::key_type;
+ using Mapped = typename Data::mapped_type;
+ static constexpr bool has_nullable_keys = has_nullable_keys_;
+
+ Data data;
+
+ AggregationMethodKeysFixed() = default;
+
+ template <typename Other>
+ AggregationMethodKeysFixed(const Other & other) : data(other.data) {}
+
+ using State = ColumnsHashing::HashMethodKeysFixed<
+ typename Data::value_type,
+ Key,
+ Mapped,
+ has_nullable_keys,
+ false,
+ use_cache>;
+#if 0
+ std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> & key_columns, const Sizes & key_sizes)
+ {
+ return State::shuffleKeyColumns(key_columns, key_sizes);
+ }
+#endif
+ template <typename ColPtr>
+ static void insertKeyIntoColumns(const Key & key, const std::vector<ColPtr> & key_columns, const Sizes & key_sizes)
+ {
+ size_t keys_count = key_columns.size();
+
+ static constexpr auto bitmap_size = has_nullable_keys ? std::tuple_size<KeysNullMap<Key>>::value : 0;
+ /// In any hash key value, column values to be read start just after the bitmap, if it exists.
+ const char * key_data = reinterpret_cast<const char *>(&key) + bitmap_size;
+
+ for (size_t i = 0; i < keys_count; ++i)
+ {
+ auto & observed_column = *key_columns[i];
+
+ if constexpr (has_nullable_keys)
+ {
+ const char * null_bitmap = reinterpret_cast<const char *>(&key);
+ size_t bucket = i / 8;
+ size_t offset = i % 8;
+ bool is_null = (null_bitmap[bucket] >> offset) & 1;
+
+ if (is_null)
+ {
+ observed_column.AppendNull().ok();
+ continue;
+ }
+ }
+
+ insertData(observed_column, StringRef(key_data, key_sizes[i]));
+ key_data += key_sizes[i];
+ }
+ }
+};
+
+
+/** Aggregates by concatenating serialized key values.
+ * The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts.
+ * That is, for example, for strings, it contains first the serialized length of the string, and then the bytes.
+ * Therefore, when aggregating by several strings, there is no ambiguity.
+ */
+template <typename TData>
+struct AggregationMethodSerialized
+{
+ using Data = TData;
+ using Key = typename Data::key_type;
+ using Mapped = typename Data::mapped_type;
+
+ Data data;
+
+ AggregationMethodSerialized() = default;
+
+ template <typename Other>
+ AggregationMethodSerialized(const Other & other) : data(other.data) {}
+
+ using State = ColumnsHashing::HashMethodSerialized<typename Data::value_type, Mapped>;
+
+ std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
+
+ template <typename ColPtr>
+ static void insertKeyIntoColumns(const StringRef & key, const std::vector<ColPtr> & key_columns, const Sizes &)
+ {
+ const auto * pos = key.data;
+ for (auto & column : key_columns)
+ pos = deserializeAndInsertFromArena(*column, pos);
+ }
+};
+
+
+class Aggregator;
+
+using ColumnsHashing::HashMethodContext;
+using ColumnsHashing::HashMethodContextPtr;
+
+struct AggregatedDataVariants //: private boost::noncopyable
+{
+ /** Working with states of aggregate functions in the pool is arranged in the following (inconvenient) way:
+ * - when aggregating, states are created in the pool using IAggregateFunction::create (inside - `placement new` of arbitrary structure);
+ * - they must then be destroyed using IAggregateFunction::destroy (inside - calling the destructor of arbitrary structure);
+ * - if aggregation is complete, then, in the Aggregator::convertToBlocks function, pointers to the states of aggregate functions
+ * are written to ColumnAggregateFunction; ColumnAggregateFunction "acquires ownership" of them, that is - calls `destroy` in its destructor.
+ * - if during the aggregation, before call to Aggregator::convertToBlocks, an exception was thrown,
+ * then the states of aggregate functions must still be destroyed,
+ * otherwise, for complex states (eg, AggregateFunctionUniq), there will be memory leaks;
+ * - in this case, to destroy states, the destructor calls Aggregator::destroyAggregateStates method,
+ * but only if the variable aggregator (see below) is not nullptr;
+ * - that is, until you transfer ownership of the aggregate function states in the ColumnAggregateFunction, set the variable `aggregator`,
+ * so that when an exception occurs, the states are correctly destroyed.
+ *
+ * PS. This can be corrected by making a pool that knows about which states of aggregate functions and in which order are put in it, and knows how to destroy them.
+ * But this can hardly be done simply because it is planned to put variable-length strings into the same pool.
+ * In this case, the pool will not be able to know with what offsets objects are stored.
+ */
+ const Aggregator * aggregator = nullptr;
+
+ size_t keys_size{}; /// Number of keys. NOTE do we need this field?
+ Sizes key_sizes; /// Dimensions of keys, if keys of fixed length
+
+ /// Pools for states of aggregate functions. Ownership will be later transferred to ColumnAggregateFunction.
+ Arenas aggregates_pools;
+ Arena * aggregates_pool{}; /// The pool that is currently used for allocation.
+
+ /** Specialization for the case when there are no keys, and for keys not fitted into max_rows_to_group_by.
+ */
+ AggregatedDataWithoutKey without_key = nullptr;
+
+ // Disable consecutive key optimization for Uint8/16, because they use a FixedHashMap
+ // and the lookup there is almost free, so we don't need to cache the last lookup result
+ std::unique_ptr<AggregationMethodOneNumber<UInt8, AggregatedDataWithUInt8Key, false>> key8;
+ std::unique_ptr<AggregationMethodOneNumber<UInt16, AggregatedDataWithUInt16Key, false>> key16;
+
+ std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64Key>> key32;
+ std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64Key>> key64;
+ std::unique_ptr<AggregationMethodStringNoCache<AggregatedDataWithShortStringKey>> key_string;
+ std::unique_ptr<AggregationMethodFixedStringNoCache<AggregatedDataWithShortStringKey>> key_fixed_string;
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithUInt16Key, false, false>> keys16;
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithUInt32Key>> keys32;
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithUInt64Key>> keys64;
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128>> keys128;
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256>> keys256;
+ std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKey>> serialized;
+
+ std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64KeyHash64>> key64_hash64;
+ std::unique_ptr<AggregationMethodString<AggregatedDataWithStringKeyHash64>> key_string_hash64;
+ std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKeyHash64>> key_fixed_string_hash64;
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128Hash64>> keys128_hash64;
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256Hash64>> keys256_hash64;
+ std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyHash64>> serialized_hash64;
+
+ /// Support for nullable keys.
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128, true>> nullable_keys128;
+ std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256, true>> nullable_keys256;
+
+ /// In this and similar macros, the option without_key is not considered.
+ #define APPLY_FOR_AGGREGATED_VARIANTS(M) \
+ M(key8) \
+ M(key16) \
+ M(key32) \
+ M(key64) \
+ M(key_string) \
+ M(key_fixed_string) \
+ M(keys16) \
+ M(keys32) \
+ M(keys64) \
+ M(keys128) \
+ M(keys256) \
+ M(serialized) \
+ M(key64_hash64) \
+ M(key_string_hash64) \
+ M(key_fixed_string_hash64) \
+ M(keys128_hash64) \
+ M(keys256_hash64) \
+ M(serialized_hash64) \
+ M(nullable_keys128) \
+ M(nullable_keys256) \
+
+
+ enum class Type
+ {
+ EMPTY = 0,
+ without_key,
+
+ #define M(NAME) NAME,
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ };
+ Type type = Type::EMPTY;
+
+ AggregatedDataVariants()
+ : aggregates_pools(1, std::make_shared<Arena>())
+ , aggregates_pool(aggregates_pools.back().get())
+ {}
+
+ bool empty() const { return type == Type::EMPTY; }
+ void invalidate() { type = Type::EMPTY; }
+
+ ~AggregatedDataVariants();
+
+ void init(Type type_)
+ {
+ switch (type_)
+ {
+ case Type::EMPTY: break;
+ case Type::without_key: break;
+
+ #define M(NAME) \
+ case Type::NAME: NAME = std::make_unique<decltype(NAME)::element_type>(); break;
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ }
+
+ type = type_;
+ }
+
+ /// Number of rows (different keys).
+ size_t size() const
+ {
+ switch (type)
+ {
+ case Type::EMPTY: return 0;
+ case Type::without_key: return 1;
+
+ #define M(NAME) \
+ case Type::NAME: return NAME->data.size() + (without_key != nullptr);
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ }
+
+ __builtin_unreachable();
+ }
+
+ /// The size without taking into account the row in which data is written for the calculation of TOTALS.
+ size_t sizeWithoutOverflowRow() const
+ {
+ switch (type)
+ {
+ case Type::EMPTY: return 0;
+ case Type::without_key: return 1;
+
+ #define M(NAME) \
+ case Type::NAME: return NAME->data.size();
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ }
+
+ __builtin_unreachable();
+ }
+
+ const char * getMethodName() const
+ {
+ switch (type)
+ {
+ case Type::EMPTY: return "EMPTY";
+ case Type::without_key: return "without_key";
+
+ #define M(NAME) \
+ case Type::NAME: return #NAME;
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ }
+
+ __builtin_unreachable();
+ }
+
+ static HashMethodContextPtr createCache(Type type, const HashMethodContext::Settings & settings)
+ {
+ switch (type)
+ {
+ case Type::without_key: return nullptr;
+
+ #define M(NAME) \
+ case Type::NAME: \
+ { \
+ using TPtr ## NAME = decltype(AggregatedDataVariants::NAME); \
+ using T ## NAME = typename TPtr ## NAME ::element_type; \
+ return T ## NAME ::State::createContext(settings); \
+ }
+
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+
+ default:
+ throw Exception("Unknown aggregated data variant.");
+ }
+ }
+};
+
+using AggregatedDataVariantsPtr = std::shared_ptr<AggregatedDataVariants>;
+using ManyAggregatedDataVariants = std::vector<AggregatedDataVariantsPtr>;
+using ManyAggregatedDataVariantsPtr = std::shared_ptr<ManyAggregatedDataVariants>;
+
+/** How are "total" values calculated with WITH TOTALS?
+ * (For more details, see TotalsHavingTransform.)
+ *
+ * In the absence of group_by_overflow_mode = 'any', the data is aggregated as usual, but the states of the aggregate functions are not finalized.
+ * Later, the aggregate function states for all rows (passed through HAVING) are merged into one - this will be TOTALS.
+ *
+ * If there is group_by_overflow_mode = 'any', the data is aggregated as usual, except for the keys that did not fit in max_rows_to_group_by.
+ * For these keys, the data is aggregated into one additional row - see below under the names `overflow_row`, `overflows`...
+ * Later, the aggregate function states for all rows (passed through HAVING) are merged into one,
+ * also overflow_row is added or not added (depending on the totals_mode setting) also - this will be TOTALS.
+ */
+
+
+/** Aggregates the source of the blocks.
+ */
+class Aggregator final
+{
+public:
+ struct Params
+ {
+ /// Data structure of source blocks.
+ Header src_header;
+ /// Data structure of intermediate blocks before merge.
+ Header intermediate_header;
+
+ /// What to count.
+ const ColumnNumbers keys;
+ const AggregateDescriptions aggregates;
+ const size_t keys_size;
+ const size_t aggregates_size;
+
+ /// The settings of approximate calculation of GROUP BY.
+ const bool overflow_row; /// Do we need to put into AggregatedDataVariants::without_key aggregates for keys that are not in max_rows_to_group_by.
+ const size_t max_rows_to_group_by = 0;
+ const OverflowMode group_by_overflow_mode = OverflowMode::THROW;
+
+ /// Return empty result when aggregating without keys on empty set.
+ bool empty_result_for_aggregation_by_empty_set = false;
+
+ /// Settings is used to determine cache size. No threads are created.
+ size_t max_threads;
+
+ Params(const Header & src_header_,
+ const Header & intermediate_header_,
+ const ColumnNumbers & keys_,
+ const AggregateDescriptions & aggregates_,
+ bool overflow_row_,
+ size_t max_threads_ = 1)
+ : src_header(src_header_)
+ , intermediate_header(intermediate_header_)
+ , keys(keys_)
+ , aggregates(aggregates_)
+ , keys_size(keys.size())
+ , aggregates_size(aggregates.size())
+ , overflow_row(overflow_row_)
+ , max_threads(max_threads_)
+ {}
+
+ Params(bool is_megre, const Header & header_,
+ const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_ = 1)
+ : Params((is_megre ? Header() : header_), (is_megre ? header_ : Header()), keys_, aggregates_, overflow_row_, max_threads_)
+ {}
+
+ static Header getHeader(
+ const Header & src_header,
+ const Header & intermediate_header,
+ const ColumnNumbers & keys,
+ const AggregateDescriptions & aggregates,
+ bool final);
+
+ Header getHeader(bool final) const
+ {
+ return getHeader(src_header, intermediate_header, keys, aggregates, final);
+ }
+ };
+
+ explicit Aggregator(const Params & params_);
+
+ /// Aggregate the source. Get the result in the form of one of the data structures.
+ void execute(const BlockInputStreamPtr & stream, AggregatedDataVariants & result);
+
+ using AggregateColumns = std::vector<ColumnRawPtrs>;
+ using AggregateFunctionsPlainPtrs = std::vector<const IAggregateFunction *>;
+
+ /// Process one block. Return false if the processing should be aborted (with group_by_overflow_mode = 'break').
+ bool executeOnBlock(const Block & block,
+ AggregatedDataVariants & result,
+ ColumnRawPtrs & key_columns,
+ AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
+ bool & no_more_keys) const;
+
+ bool executeOnBlock(Columns columns,
+ size_t row_begin, size_t row_end,
+ AggregatedDataVariants & result,
+ ColumnRawPtrs & key_columns,
+ AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
+ bool & no_more_keys) const;
+
+ /// Used for aggregate projection.
+ bool mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const;
+
+ /** Convert the aggregation data structure into a block.
+ * If overflow_row = true, then aggregates for rows that are not included in max_rows_to_group_by are put in the first block.
+ *
+ * If final = false, then ColumnAggregateFunction is created as the aggregation columns with the state of the calculations,
+ * which can then be combined with other states (for distributed query processing).
+ * If final = true, then columns with ready values are created as aggregate columns.
+ */
+ BlocksList convertToBlocks(AggregatedDataVariants & data_variants, bool final) const;
+
+ ManyAggregatedDataVariants prepareVariantsToMerge(ManyAggregatedDataVariants & data_variants) const;
+
+ /** Merge the stream of partially aggregated blocks into one data structure.
+ * (Pre-aggregate several blocks that represent the result of independent aggregations from remote servers.)
+ */
+ void mergeStream(const BlockInputStreamPtr & stream, AggregatedDataVariants & result);
+
+ using BucketToBlocks = std::map<Int32, BlocksList>;
+ /// Merge partially aggregated blocks separated to buckets into one data structure.
+ void mergeBlocks(BucketToBlocks && bucket_to_blocks, AggregatedDataVariants & result);
+
+ /// Merge several partially aggregated blocks into one.
+ /// Precondition: for all blocks block.info.is_overflows flag must be the same.
+ /// (either all blocks are from overflow data or none blocks are).
+ /// The resulting block has the same value of is_overflows flag.
+ Block mergeBlocks(BlocksList & blocks, bool final);
+
+ /// Get data structure of the result.
+ Header getHeader(bool final) const;
+
+private:
+ friend struct AggregatedDataVariants;
+ friend class MergingAndConvertingBlockInputStream;
+
+ Params params;
+
+ AggregatedDataVariants::Type method_chosen;
+ Sizes key_sizes;
+
+ HashMethodContextPtr aggregation_state_cache;
+
+ AggregateFunctionsPlainPtrs aggregate_functions;
+
+ /** This array serves two purposes.
+ *
+ * Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated.
+ * The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%.
+ */
+ struct AggregateFunctionInstruction
+ {
+ const IAggregateFunction * that{};
+ size_t state_offset{};
+ const IColumn ** arguments{};
+ const IAggregateFunction * batch_that{};
+ const IColumn ** batch_arguments{};
+ };
+
+ using AggregateFunctionInstructions = std::vector<AggregateFunctionInstruction>;
+
+ Sizes offsets_of_aggregate_states; /// The offset to the n-th aggregate function in a row of aggregate functions.
+ size_t total_size_of_aggregate_states = 0; /// The total size of the row from the aggregate functions.
+
+ // add info to track alignment requirement
+ // If there are states whose alignment are v1, ..vn, align_aggregate_states will be max(v1, ... vn)
+ size_t align_aggregate_states = 1;
+
+ bool all_aggregates_has_trivial_destructor = false;
+
+ /** Select the aggregation method based on the number and types of keys. */
+ AggregatedDataVariants::Type chooseAggregationMethod();
+
+ /** Create states of aggregate functions for one key.
+ */
+ void createAggregateStates(AggregateDataPtr & aggregate_data) const;
+
+ /** Call `destroy` methods for states of aggregate functions.
+ * Used in the exception handler for aggregation, since RAII in this case is not applicable.
+ */
+ void destroyAllAggregateStates(AggregatedDataVariants & result) const;
+
+
+ /// Process one data block, aggregate the data into a hash table.
+ template <typename Method>
+ void executeImpl(
+ Method & method,
+ Arena * aggregates_pool,
+ size_t row_begin,
+ size_t row_end,
+ ColumnRawPtrs & key_columns,
+ AggregateFunctionInstruction * aggregate_instructions,
+ bool no_more_keys,
+ AggregateDataPtr overflow_row) const;
+
+ /// Specialization for a particular value no_more_keys.
+ template <bool no_more_keys, typename Method>
+ void executeImplBatch(
+ Method & method,
+ typename Method::State & state,
+ Arena * aggregates_pool,
+ size_t row_begin,
+ size_t row_end,
+ AggregateFunctionInstruction * aggregate_instructions,
+ AggregateDataPtr overflow_row) const;
+
+ /// For case when there are no keys (all aggregate into one row).
+ void executeWithoutKeyImpl(
+ AggregatedDataWithoutKey & res,
+ size_t row_begin,
+ size_t row_end,
+ AggregateFunctionInstruction * aggregate_instructions,
+ Arena * arena) const;
+
+ /// Merge NULL key data from hash table `src` into `dst`.
+ template <typename Method, typename Table>
+ void mergeDataNullKey(
+ Table & table_dst,
+ Table & table_src,
+ Arena * arena) const;
+
+ /// Merge data from hash table `src` into `dst`.
+ template <typename Method, typename Table>
+ void mergeDataImpl(
+ Table & table_dst,
+ Table & table_src,
+ Arena * arena) const;
+
+ /// Merge data from hash table `src` into `dst`, but only for keys that already exist in dst. In other cases, merge the data into `overflows`.
+ template <typename Method, typename Table>
+ void mergeDataNoMoreKeysImpl(
+ Table & table_dst,
+ AggregatedDataWithoutKey & overflows,
+ Table & table_src,
+ Arena * arena) const;
+
+ /// Same, but ignores the rest of the keys.
+ template <typename Method, typename Table>
+ void mergeDataOnlyExistingKeysImpl(
+ Table & table_dst,
+ Table & table_src,
+ Arena * arena) const;
+
+ void mergeWithoutKeyDataImpl(
+ ManyAggregatedDataVariants & non_empty_data) const;
+
+ template <typename Method>
+ void mergeSingleLevelDataImpl(
+ ManyAggregatedDataVariants & non_empty_data) const;
+
+ template <typename Method, typename Table>
+ void convertToBlockImpl(
+ Method & method,
+ Table & data,
+ MutableColumns & key_columns,
+ AggregateColumnsData & aggregate_columns,
+ MutableColumns & final_aggregate_columns,
+ Arena * arena,
+ bool final) const;
+
+ template <typename Mapped>
+ void insertAggregatesIntoColumns(
+ Mapped & mapped,
+ MutableColumns & final_aggregate_columns,
+ Arena * arena) const;
+
+ template <typename Method, typename Table>
+ void convertToBlockImplFinal(
+ Method & method,
+ Table & data,
+ const MutableColumns & key_columns,
+ MutableColumns & final_aggregate_columns,
+ Arena * arena) const;
+
+ template <typename Method, typename Table>
+ void convertToBlockImplNotFinal(
+ Method & method,
+ Table & data,
+ const MutableColumns & key_columns,
+ AggregateColumnsData & aggregate_columns) const;
+
+ template <typename Filler>
+ Block prepareBlockAndFill(
+ AggregatedDataVariants & data_variants,
+ bool final,
+ size_t rows,
+ Filler && filler) const;
+
+ template <typename Method>
+ Block convertOneBucketToBlock(
+ AggregatedDataVariants & data_variants,
+ Method & method,
+ Arena * arena,
+ bool final,
+ size_t bucket) const;
+
+ Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows = false) const;
+ Block prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const;
+
+ template <bool no_more_keys, typename Method, typename Table>
+ void mergeStreamsImplCase(
+ Block & block,
+ Arena * aggregates_pool,
+ Method & method,
+ Table & data,
+ AggregateDataPtr overflow_row) const;
+
+ template <typename Method, typename Table>
+ void mergeStreamsImpl(
+ Block & block,
+ Arena * aggregates_pool,
+ Method & method,
+ Table & data,
+ AggregateDataPtr overflow_row,
+ bool no_more_keys) const;
+
+ void mergeWithoutKeyStreamsImpl(
+ Block & block,
+ AggregatedDataVariants & result) const;
+
+ template <typename Method>
+ void mergeBucketImpl(
+ ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena, std::atomic<bool> * is_cancelled = nullptr) const;
+
+ template <typename Method>
+ void convertBlockToTwoLevelImpl(
+ Method & method,
+ Arena * pool,
+ ColumnRawPtrs & key_columns,
+ const Block & source,
+ std::vector<Block> & destinations) const;
+
+ template <typename Method, typename Table>
+ void destroyImpl(Table & table) const;
+
+ void destroyWithoutKey(
+ AggregatedDataVariants & result) const;
+
+
+ /** Checks constraints on the maximum number of keys for aggregation.
+ * If it is exceeded, then, depending on the group_by_overflow_mode, either
+ * - throws an exception;
+ * - returns false, which means that execution must be aborted;
+ * - sets the variable no_more_keys to true.
+ */
+ bool checkLimits(size_t result_size, bool & no_more_keys) const;
+
+ void prepareAggregateInstructions(
+ Columns columns,
+ AggregateColumns & aggregate_columns,
+ Columns & materialized_columns,
+ AggregateFunctionInstructions & instructions) const;
+};
+
+
+/** Get the aggregation variant by its type. */
+template <typename Method> Method & getDataVariant(AggregatedDataVariants & variants);
+
+#define M(NAME) \
+ template <> inline decltype(AggregatedDataVariants::NAME)::element_type & getDataVariant<decltype(AggregatedDataVariants::NAME)::element_type>(AggregatedDataVariants & variants) { return *variants.NAME; }
+
+APPLY_FOR_AGGREGATED_VARIANTS(M)
+
+#undef M
+
+}
diff --git a/ydb/library/arrow_clickhouse/CMakeLists.txt b/ydb/library/arrow_clickhouse/CMakeLists.txt
new file mode 100644
index 00000000000..97a7a6b939f
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/CMakeLists.txt
@@ -0,0 +1,29 @@
+
+# This file was gererated by the build system used internally in the Yandex monorepo.
+# Only simple modifications are allowed (adding source-files to targets, adding simple properties
+# like target_include_directories). These modifications will be ported to original
+# ya.make files by maintainers. Any complex modifications which can't be ported back to the
+# original buildsystem will not be accepted.
+
+
+
+add_library(ydb-library-arrow_clickhouse)
+target_include_directories(ydb-library-arrow_clickhouse PUBLIC
+ ${CMAKE_SOURCE_DIR}/ydb/library/yql/udfs/common/clickhouse/client/base
+)
+target_include_directories(ydb-library-arrow_clickhouse PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse
+)
+target_link_libraries(ydb-library-arrow_clickhouse PUBLIC
+ contrib-libs-cxxsupp
+ yutil
+ libs-apache-arrow
+ library-arrow_clickhouse-Common
+ library-arrow_clickhouse-Columns
+ library-arrow_clickhouse-DataStreams
+)
+target_sources(ydb-library-arrow_clickhouse PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/Aggregator.cpp
+ ${CMAKE_SOURCE_DIR}/ydb/library/yql/udfs/common/clickhouse/client/base/common/mremap.cpp
+ ${CMAKE_SOURCE_DIR}/ydb/library/yql/udfs/common/clickhouse/client/base/common/getPageSize.cpp
+)
diff --git a/ydb/library/arrow_clickhouse/Columns/CMakeLists.txt b/ydb/library/arrow_clickhouse/Columns/CMakeLists.txt
new file mode 100644
index 00000000000..0f4c153a983
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Columns/CMakeLists.txt
@@ -0,0 +1,23 @@
+
+# This file was gererated by the build system used internally in the Yandex monorepo.
+# Only simple modifications are allowed (adding source-files to targets, adding simple properties
+# like target_include_directories). These modifications will be ported to original
+# ya.make files by maintainers. Any complex modifications which can't be ported back to the
+# original buildsystem will not be accepted.
+
+
+
+add_library(library-arrow_clickhouse-Columns)
+target_include_directories(library-arrow_clickhouse-Columns PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/yql/udfs/common/clickhouse/client/base
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse
+)
+target_link_libraries(library-arrow_clickhouse-Columns PUBLIC
+ contrib-libs-cxxsupp
+ yutil
+ libs-apache-arrow
+)
+target_sources(library-arrow_clickhouse-Columns PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/Columns/ColumnsCommon.cpp
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.cpp
+)
diff --git a/ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.cpp b/ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.cpp
new file mode 100644
index 00000000000..7d854890f3c
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.cpp
@@ -0,0 +1,31 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#include <sstream>
+
+#include <Columns/ColumnAggregateFunction.h>
+#include <Columns/ColumnsCommon.h>
+#include <Common/Arena.h>
+#include <Common/HashTable/Hash.h>
+
+
+namespace CH
+{
+
+std::shared_ptr<arrow::Array> DataTypeAggregateFunction::MakeArray(std::shared_ptr<arrow::ArrayData> data) const
+{
+ return std::make_shared<ColumnAggregateFunction>(data);
+}
+
+ColumnAggregateFunction::~ColumnAggregateFunction()
+{
+ if (!func->hasTrivialDestructor() && !src)
+ {
+ auto & arr = getData();
+ for (int64_t i = 0; i < arr.length(); ++i)
+ func->destroy(reinterpret_cast<AggregateDataPtr>(arr.Value(i)));
+ }
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.h b/ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.h
new file mode 100644
index 00000000000..cf126113230
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Columns/ColumnAggregateFunction.h
@@ -0,0 +1,150 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <AggregateFunctions/IAggregateFunction.h>
+
+#include <common/StringRef.h>
+#include <ranges>
+
+namespace CH
+{
+
+class DataTypeAggregateFunction final : public arrow::ExtensionType
+{
+public:
+ static constexpr const char * FAMILY_NAME = "aggregate_function";
+
+ DataTypeAggregateFunction(const AggregateFunctionPtr & function_,
+ const DataTypes & argument_types_,
+ const Array & parameters_)
+ : arrow::ExtensionType(arrow::uint64())
+ , function(function_)
+ , argument_types(argument_types_)
+ , parameters(parameters_)
+ {}
+
+ std::string extension_name() const override { return FAMILY_NAME; }
+
+ bool ExtensionEquals(const arrow::ExtensionType& other) const override
+ {
+ return extension_name() != other.extension_name(); // TODO
+ }
+
+ std::shared_ptr<arrow::Array> MakeArray(std::shared_ptr<arrow::ArrayData> data) const override;
+
+ virtual arrow::Result<std::shared_ptr<arrow::DataType>> Deserialize(std::shared_ptr<arrow::DataType> /*storage_type*/,
+ const std::string& /*serialized_data*/) const override
+ {
+ return std::make_shared<DataTypeAggregateFunction>(AggregateFunctionPtr{}, DataTypes{}, Array{}); // TODO
+ }
+
+ std::string Serialize() const override { return {}; } // TODO
+
+ AggregateFunctionPtr getFunction() const { return function; }
+
+private:
+ AggregateFunctionPtr function;
+ DataTypes argument_types;
+ Array parameters;
+};
+
+/** Column of states of aggregate functions.
+ * Presented as an array of pointers to the states of aggregate functions (data).
+ * The states themselves are stored in one of the pools (arenas).
+ *
+ * It can be in two variants:
+ *
+ * 1. Own its values - that is, be responsible for destroying them.
+ * The column consists of the values "assigned to it" after the aggregation is performed (see Aggregator, convertToBlocks function),
+ * or from values created by itself (see `insert` method).
+ * In this case, `src` will be `nullptr`, and the column itself will be destroyed (call `IAggregateFunction::destroy`)
+ * states of aggregate functions in the destructor.
+ *
+ * 2. Do not own its values, but use values taken from another ColumnAggregateFunction column.
+ * For example, this is a column obtained by permutation/filtering or other transformations from another column.
+ * In this case, `src` will be `shared ptr` to the source column. Destruction of values will be handled by this source column.
+ *
+ * This solution is somewhat limited:
+ * - the variant in which the column contains a part of "it's own" and a part of "another's" values is not supported;
+ * - the option of having multiple source columns is not supported, which may be necessary for a more optimal merge of the two columns.
+ *
+ * These restrictions can be removed if you add an array of flags or even refcount,
+ * specifying which individual values should be destroyed and which ones should not.
+ * Clearly, this method would have a substantially non-zero price.
+ */
+class ColumnAggregateFunction final : public arrow::ExtensionArray
+{
+private:
+#if 0
+ /// Arenas used by function states that are created elsewhere. We own these
+ /// arenas in the sense of extending their lifetime, but do not modify them.
+ /// Even reading these arenas is unsafe, because they may be shared with
+ /// other data blocks and modified by other threads concurrently.
+ ConstArenas foreign_arenas;
+#endif
+ /// Used for destroying states and for finalization of values.
+ AggregateFunctionPtr func;
+
+ /// Source column. Used (holds source from destruction),
+ /// if this column has been constructed from another and uses all or part of its values.
+ ColumnPtr src;
+
+public:
+ ColumnAggregateFunction(const std::shared_ptr<DataTypeAggregateFunction> & data_type)
+ : arrow::ExtensionArray(data_type, *arrow::MakeArrayOfNull(arrow::uint64(), 0))
+ , func(data_type->getFunction())
+ {}
+
+ explicit ColumnAggregateFunction(const std::shared_ptr<arrow::ArrayData>& data)
+ : arrow::ExtensionArray(data)
+ , func(std::static_pointer_cast<DataTypeAggregateFunction>(data->type)->getFunction())
+ {}
+
+ ~ColumnAggregateFunction() override;
+
+ const arrow::UInt64Array & getData() const { return static_cast<arrow::UInt64Array &>(*storage()); }
+ const AggregateDataPtr * rawData() const { return reinterpret_cast<const AggregateDataPtr *>(getData().raw_values()); }
+};
+
+
+class MutableColumnAggregateFunction final : public arrow::ArrayBuilder
+{
+public:
+ MutableColumnAggregateFunction(const std::shared_ptr<DataTypeAggregateFunction> & data_type_,
+ arrow::MemoryPool* pool = arrow::default_memory_pool())
+ : arrow::ArrayBuilder(pool)
+ , data_type(data_type_)
+ , builder(std::make_shared<arrow::UInt64Builder>(pool))
+ {}
+
+ std::shared_ptr<arrow::DataType> type() const override { return data_type; }
+
+ arrow::Status AppendNull() override { return arrow::Status(arrow::StatusCode::NotImplemented, __FUNCTION__); }
+ arrow::Status AppendNulls(int64_t) override { return arrow::Status(arrow::StatusCode::NotImplemented, __FUNCTION__); }
+ arrow::Status AppendEmptyValue() override { return arrow::Status(arrow::StatusCode::NotImplemented, __FUNCTION__); }
+ arrow::Status AppendEmptyValues(int64_t) override { return arrow::Status(arrow::StatusCode::NotImplemented, __FUNCTION__); }
+
+ arrow::Status FinishInternal(std::shared_ptr<arrow::ArrayData>* out) override
+ {
+ auto array = *builder->Finish();
+ *out = array->data()->Copy();
+ (*out)->type = data_type;
+ // TODO: add arenas
+ return arrow::Status::OK();
+ }
+
+ arrow::UInt64Builder & getData() { return *builder; }
+
+private:
+ std::shared_ptr<DataTypeAggregateFunction> data_type;
+ std::shared_ptr<arrow::UInt64Builder> builder;
+};
+
+using AggregateColumnsData = std::vector<arrow::UInt64Builder *>;
+using AggregateColumnsConstData = std::vector<const arrow::UInt64Array *>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Columns/ColumnsCommon.cpp b/ydb/library/arrow_clickhouse/Columns/ColumnsCommon.cpp
new file mode 100644
index 00000000000..3dbc50b22ff
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Columns/ColumnsCommon.cpp
@@ -0,0 +1,530 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#ifdef __SSE2__
+ #include <emmintrin.h>
+#endif
+
+#include <Columns/ColumnsCommon.h>
+#include <Common/HashTable/HashSet.h>
+#include <Common/PODArray.h>
+
+
+namespace CH
+{
+
+#if defined(__SSE2__) && defined(__POPCNT__)
+/// Transform 64-byte mask to 64-bit mask.
+static UInt64 toBits64(const Int8 * bytes64)
+{
+ static const __m128i zero16 = _mm_setzero_si128();
+ UInt64 res =
+ static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpeq_epi8(
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(bytes64)), zero16)))
+ | (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpeq_epi8(
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(bytes64 + 16)), zero16))) << 16)
+ | (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpeq_epi8(
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(bytes64 + 32)), zero16))) << 32)
+ | (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpeq_epi8(
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(bytes64 + 48)), zero16))) << 48);
+
+ return ~res;
+}
+#endif
+
+size_t countBytesInFilter(const uint8_t * filt, size_t start, size_t end)
+{
+ size_t count = 0;
+
+ /** NOTE: In theory, `filt` should only contain zeros and ones.
+ * But, just in case, here the condition > 0 (to signed bytes) is used.
+ * It would be better to use != 0, then this does not allow SSE2.
+ */
+
+ const Int8 * pos = reinterpret_cast<const Int8 *>(filt);
+ pos += start;
+
+ const Int8 * end_pos = pos + (end - start);
+
+#if defined(__SSE2__)
+ const Int8 * end_pos64 = pos + (end - start) / 64 * 64;
+
+ for (; pos < end_pos64; pos += 64)
+ count += std::popcount(toBits64(pos));
+
+ /// TODO Add duff device for tail?
+#endif
+
+ for (; pos < end_pos; ++pos)
+ count += *pos != 0;
+
+ return count;
+}
+
+size_t countBytesInFilterWithNull(const uint8_t * filt, const uint8_t * null_map, size_t start, size_t end)
+{
+ size_t count = 0;
+
+ /** NOTE: In theory, `filt` should only contain zeros and ones.
+ * But, just in case, here the condition > 0 (to signed bytes) is used.
+ * It would be better to use != 0, then this does not allow SSE2.
+ */
+
+ const Int8 * pos = reinterpret_cast<const Int8 *>(filt) + start;
+ const Int8 * pos2 = reinterpret_cast<const Int8 *>(null_map) + start;
+ const Int8 * end_pos = pos + (end - start);
+
+#if defined(__SSE2__)
+ const Int8 * end_pos64 = pos + (end - start) / 64 * 64;
+
+ for (; pos < end_pos64; pos += 64, pos2 += 64)
+ count += std::popcount(toBits64(pos) & ~toBits64(pos2));
+
+ /// TODO Add duff device for tail?
+#endif
+
+ for (; pos < end_pos; ++pos, ++pos2)
+ count += (*pos & ~*pos2) != 0;
+
+ return count;
+}
+
+namespace
+{
+ /// Implementation details of filterArraysImpl function, used as template parameter.
+ /// Allow to build or not to build offsets array.
+
+ struct ResultOffsetsBuilder
+ {
+ PaddedPODArray<UInt64> & res_offsets;
+ XColumn::Offset current_src_offset = 0;
+
+ explicit ResultOffsetsBuilder(PaddedPODArray<UInt64> * res_offsets_) : res_offsets(*res_offsets_) {}
+
+ void reserve(ssize_t result_size_hint, size_t src_size)
+ {
+ res_offsets.reserve(result_size_hint > 0 ? result_size_hint : src_size);
+ }
+
+ void insertOne(size_t array_size)
+ {
+ current_src_offset += array_size;
+ res_offsets.push_back(current_src_offset);
+ }
+
+ template <size_t SIMD_BYTES>
+ void insertChunk(
+ const XColumn::Offset * src_offsets_pos,
+ bool first,
+ XColumn::Offset chunk_offset,
+ size_t chunk_size)
+ {
+ const auto offsets_size_old = res_offsets.size();
+ res_offsets.resize(offsets_size_old + SIMD_BYTES);
+ memcpy(&res_offsets[offsets_size_old], src_offsets_pos, SIMD_BYTES * sizeof(XColumn::Offset));
+
+ if (!first)
+ {
+ /// difference between current and actual offset
+ const auto diff_offset = chunk_offset - current_src_offset;
+
+ if (diff_offset > 0)
+ {
+ auto * res_offsets_pos = &res_offsets[offsets_size_old];
+
+ /// adjust offsets
+ for (size_t i = 0; i < SIMD_BYTES; ++i)
+ res_offsets_pos[i] -= diff_offset;
+ }
+ }
+ current_src_offset += chunk_size;
+ }
+ };
+
+ struct NoResultOffsetsBuilder
+ {
+ explicit NoResultOffsetsBuilder(PaddedPODArray<UInt64> *) {}
+ void reserve(ssize_t, size_t) {}
+ void insertOne(size_t) {}
+
+ template <size_t SIMD_BYTES>
+ void insertChunk(
+ const XColumn::Offset *,
+ bool,
+ XColumn::Offset,
+ size_t)
+ {
+ }
+ };
+
+
+ template <typename T, typename ResultOffsetsBuilder>
+ void filterArraysImplGeneric(
+ const PaddedPODArray<T> & src_elems, const PaddedPODArray<UInt64> & src_offsets,
+ PaddedPODArray<T> & res_elems, PaddedPODArray<UInt64> * res_offsets,
+ const XColumn::Filter & filt, ssize_t result_size_hint)
+ {
+ const size_t size = src_offsets.size();
+ if (size != filt.size())
+ throw Exception("Size of filter doesn't match size of column.");
+
+ ResultOffsetsBuilder result_offsets_builder(res_offsets);
+
+ if (result_size_hint)
+ {
+ result_offsets_builder.reserve(result_size_hint, size);
+
+ if (result_size_hint < 0)
+ res_elems.reserve(src_elems.size());
+ else if (result_size_hint < 1000000000 && src_elems.size() < 1000000000) /// Avoid overflow.
+ res_elems.reserve((result_size_hint * src_elems.size() + size - 1) / size);
+ }
+
+ const UInt8 * filt_pos = filt.data();
+ const auto * filt_end = filt_pos + size;
+
+ const auto * offsets_pos = src_offsets.data();
+ const auto * offsets_begin = offsets_pos;
+
+ /// copy array ending at *end_offset_ptr
+ const auto copy_array = [&] (const XColumn::Offset * offset_ptr)
+ {
+ const auto arr_offset = offset_ptr == offsets_begin ? 0 : offset_ptr[-1];
+ const auto arr_size = *offset_ptr - arr_offset;
+
+ result_offsets_builder.insertOne(arr_size);
+
+ const auto elems_size_old = res_elems.size();
+ res_elems.resize(elems_size_old + arr_size);
+ memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T));
+ };
+
+ #ifdef __SSE2__
+ const __m128i zero_vec = _mm_setzero_si128();
+ static constexpr size_t SIMD_BYTES = 16;
+ const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES;
+
+ while (filt_pos < filt_end_aligned)
+ {
+ UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(filt_pos)),
+ zero_vec));
+ mask = ~mask;
+
+ if (mask == 0)
+ {
+ /// SIMD_BYTES consecutive rows do not pass the filter
+ }
+ else if (mask == 0xffff)
+ {
+ /// SIMD_BYTES consecutive rows pass the filter
+ const auto first = offsets_pos == offsets_begin;
+
+ const auto chunk_offset = first ? 0 : offsets_pos[-1];
+ const auto chunk_size = offsets_pos[SIMD_BYTES - 1] - chunk_offset;
+
+ result_offsets_builder.template insertChunk<SIMD_BYTES>(offsets_pos, first, chunk_offset, chunk_size);
+
+ /// copy elements for SIMD_BYTES arrays at once
+ const auto elems_size_old = res_elems.size();
+ res_elems.resize(elems_size_old + chunk_size);
+ memcpy(&res_elems[elems_size_old], &src_elems[chunk_offset], chunk_size * sizeof(T));
+ }
+ else
+ {
+ for (size_t i = 0; i < SIMD_BYTES; ++i)
+ if (filt_pos[i])
+ copy_array(offsets_pos + i);
+ }
+
+ filt_pos += SIMD_BYTES;
+ offsets_pos += SIMD_BYTES;
+ }
+ #endif
+
+ while (filt_pos < filt_end)
+ {
+ if (*filt_pos)
+ copy_array(offsets_pos);
+
+ ++filt_pos;
+ ++offsets_pos;
+ }
+ }
+}
+
+bool insertData(MutableColumn & column, const StringRef & value)
+{
+ switch (column.type()->id())
+ {
+ case arrow::Type::UINT8:
+ return insertNumber(column, *reinterpret_cast<const UInt8 *>(value.data));
+ case arrow::Type::UINT16:
+ return insertNumber(column, *reinterpret_cast<const UInt16 *>(value.data));
+ case arrow::Type::UINT32:
+ return insertNumber(column, *reinterpret_cast<const UInt32 *>(value.data));
+ case arrow::Type::UINT64:
+ return insertNumber(column, *reinterpret_cast<const UInt64 *>(value.data));
+
+ case arrow::Type::INT8:
+ return insertNumber(column, *reinterpret_cast<const Int8 *>(value.data));
+ case arrow::Type::INT16:
+ return insertNumber(column, *reinterpret_cast<const Int16 *>(value.data));
+ case arrow::Type::INT32:
+ return insertNumber(column, *reinterpret_cast<const Int32 *>(value.data));
+ case arrow::Type::INT64:
+ return insertNumber(column, *reinterpret_cast<const Int64 *>(value.data));
+
+ case arrow::Type::FLOAT:
+ return insertNumber(column, *reinterpret_cast<const float *>(value.data));
+ case arrow::Type::DOUBLE:
+ return insertNumber(column, *reinterpret_cast<const double *>(value.data));
+
+ case arrow::Type::FIXED_SIZE_BINARY:
+ return insertFixedString(column, value);
+
+ case arrow::Type::STRING:
+ case arrow::Type::BINARY:
+ return insertString(column, value);
+
+ case arrow::Type::EXTENSION: // AggregateColumn
+ break; // TODO
+
+ default:
+ break;
+ }
+
+ throw Exception(std::string(__FUNCTION__) + " unexpected type " + column.type()->ToString());
+}
+
+StringRef serializeValueIntoArena(const IColumn& column, size_t row, Arena & pool, char const *& begin)
+{
+ switch (column.type_id())
+ {
+ case arrow::Type::UINT8:
+ return serializeNumberIntoArena(assert_cast<const ColumnUInt8 &>(column).Value(row), pool, begin);
+ case arrow::Type::UINT16:
+ return serializeNumberIntoArena(assert_cast<const ColumnUInt16 &>(column).Value(row), pool, begin);
+ case arrow::Type::UINT32:
+ return serializeNumberIntoArena(assert_cast<const ColumnUInt32 &>(column).Value(row), pool, begin);
+ case arrow::Type::UINT64:
+ return serializeNumberIntoArena(assert_cast<const ColumnUInt64 &>(column).Value(row), pool, begin);
+
+ case arrow::Type::INT8:
+ return serializeNumberIntoArena(assert_cast<const ColumnInt8 &>(column).Value(row), pool, begin);
+ case arrow::Type::INT16:
+ return serializeNumberIntoArena(assert_cast<const ColumnInt16 &>(column).Value(row), pool, begin);
+ case arrow::Type::INT32:
+ return serializeNumberIntoArena(assert_cast<const ColumnInt32 &>(column).Value(row), pool, begin);
+ case arrow::Type::INT64:
+ return serializeNumberIntoArena(assert_cast<const ColumnInt64 &>(column).Value(row), pool, begin);
+
+ case arrow::Type::FLOAT:
+ return serializeNumberIntoArena(assert_cast<const ColumnFloat32 &>(column).Value(row), pool, begin);
+ case arrow::Type::DOUBLE:
+ return serializeNumberIntoArena(assert_cast<const ColumnFloat64 &>(column).Value(row), pool, begin);
+
+ case arrow::Type::FIXED_SIZE_BINARY:
+ {
+ auto str = assert_cast<const ColumnFixedString &>(column).GetView(row);
+ return serializeStringIntoArena<true>(StringRef(str.data(), str.size()), pool, begin);
+ }
+ case arrow::Type::STRING:
+ case arrow::Type::BINARY:
+ {
+ auto str = assert_cast<const ColumnBinary &>(column).GetView(row);
+ return serializeStringIntoArena<false>(StringRef(str.data(), str.size()), pool, begin);
+ }
+
+ case arrow::Type::EXTENSION: // AggregateColumn
+ break; // TODO
+
+ default:
+ break;
+ }
+
+ throw Exception(std::string(__FUNCTION__) + " unexpected type " + column.type()->ToString());
+}
+
+const char * deserializeAndInsertFromArena(MutableColumn& column, const char * pos)
+{
+ switch (column.type()->id())
+ {
+ case arrow::Type::UINT8:
+ return deserializeNumberFromArena(assert_cast<MutableColumnUInt8 &>(column), pos);
+ case arrow::Type::UINT16:
+ return deserializeNumberFromArena(assert_cast<MutableColumnUInt16 &>(column), pos);
+ case arrow::Type::UINT32:
+ return deserializeNumberFromArena(assert_cast<MutableColumnUInt32 &>(column), pos);
+ case arrow::Type::UINT64:
+ return deserializeNumberFromArena(assert_cast<MutableColumnUInt64 &>(column), pos);
+
+ case arrow::Type::INT8:
+ return deserializeNumberFromArena(assert_cast<MutableColumnInt8 &>(column), pos);
+ case arrow::Type::INT16:
+ return deserializeNumberFromArena(assert_cast<MutableColumnInt16 &>(column), pos);
+ case arrow::Type::INT32:
+ return deserializeNumberFromArena(assert_cast<MutableColumnInt32 &>(column), pos);
+ case arrow::Type::INT64:
+ return deserializeNumberFromArena(assert_cast<MutableColumnInt64 &>(column), pos);
+
+ case arrow::Type::FLOAT:
+ return deserializeNumberFromArena(assert_cast<MutableColumnFloat32 &>(column), pos);
+ case arrow::Type::DOUBLE:
+ return deserializeNumberFromArena(assert_cast<MutableColumnFloat64 &>(column), pos);
+
+ case arrow::Type::FIXED_SIZE_BINARY:
+ return deserializeStringFromArena(assert_cast<MutableColumnFixedString &>(column), pos);
+
+ case arrow::Type::STRING:
+ case arrow::Type::BINARY:
+ return deserializeStringFromArena(assert_cast<MutableColumnBinary &>(column), pos);
+
+ case arrow::Type::EXTENSION: // AggregateColumn
+ break; // TODO
+
+ default:
+ break;
+ }
+
+ throw Exception(std::string(__FUNCTION__) + " unexpected type " + column.type()->ToString());
+}
+
+void updateHashWithValue(const IColumn& column, size_t row, SipHash & hash)
+{
+ switch (column.type_id())
+ {
+ case arrow::Type::UINT8:
+ return hash.update(assert_cast<const ColumnUInt8 &>(column).Value(row));
+ case arrow::Type::UINT16:
+ return hash.update(assert_cast<const ColumnUInt16 &>(column).Value(row));
+ case arrow::Type::UINT32:
+ return hash.update(assert_cast<const ColumnUInt32 &>(column).Value(row));
+ case arrow::Type::UINT64:
+ return hash.update(assert_cast<const ColumnUInt64 &>(column).Value(row));
+
+ case arrow::Type::INT8:
+ return hash.update(assert_cast<const ColumnInt8 &>(column).Value(row));
+ case arrow::Type::INT16:
+ return hash.update(assert_cast<const ColumnInt16 &>(column).Value(row));
+ case arrow::Type::INT32:
+ return hash.update(assert_cast<const ColumnInt32 &>(column).Value(row));
+ case arrow::Type::INT64:
+ return hash.update(assert_cast<const ColumnInt64 &>(column).Value(row));
+
+ case arrow::Type::FLOAT:
+ return hash.update(assert_cast<const ColumnFloat32 &>(column).Value(row));
+ case arrow::Type::DOUBLE:
+ return hash.update(assert_cast<const ColumnFloat64 &>(column).Value(row));
+
+ case arrow::Type::FIXED_SIZE_BINARY:
+ {
+ auto str = assert_cast<const ColumnFixedString &>(column).GetView(row);
+ return hash.update(str.data(), str.size());
+ }
+ case arrow::Type::STRING:
+ case arrow::Type::BINARY:
+ {
+ auto str = assert_cast<const ColumnBinary &>(column).GetView(row);
+ return hash.update(str.data(), str.size());
+ }
+
+ case arrow::Type::EXTENSION: // AggregateColumn
+ break; // TODO
+
+ default:
+ break;
+ }
+
+ throw Exception(std::string(__FUNCTION__) + " unexpected type " + column.type()->ToString());
+}
+
+MutableColumnPtr createMutableColumn(const DataTypePtr & type)
+{
+ switch (type->id())
+ {
+ case arrow::Type::UINT8:
+ return std::make_shared<MutableColumnUInt8>();
+ case arrow::Type::UINT16:
+ return std::make_shared<MutableColumnUInt16>();
+ case arrow::Type::UINT32:
+ return std::make_shared<MutableColumnUInt32>();
+ case arrow::Type::UINT64:
+ return std::make_shared<MutableColumnUInt64>();
+
+ case arrow::Type::INT8:
+ return std::make_shared<MutableColumnInt8>();
+ case arrow::Type::INT16:
+ return std::make_shared<MutableColumnInt16>();
+ case arrow::Type::INT32:
+ return std::make_shared<MutableColumnInt32>();
+ case arrow::Type::INT64:
+ return std::make_shared<MutableColumnInt64>();
+
+ case arrow::Type::FLOAT:
+ return std::make_shared<MutableColumnFloat32>();
+ case arrow::Type::DOUBLE:
+ return std::make_shared<MutableColumnFloat64>();
+
+ case arrow::Type::FIXED_SIZE_BINARY:
+ return std::make_shared<MutableColumnFixedString>(type);
+
+ case arrow::Type::BINARY:
+ return std::make_shared<MutableColumnBinary>();
+ case arrow::Type::STRING:
+ return std::make_shared<MutableColumnString>();
+
+ case arrow::Type::EXTENSION: // AggregateColumn
+ break; // TODO: do we really need it here?
+
+ default:
+ break;
+ }
+
+ throw Exception(std::string(__FUNCTION__) + " unexpected type " + type->ToString());
+}
+
+uint32_t fixedContiguousSize(const DataTypePtr & type)
+{
+ switch (type->id())
+ {
+ case arrow::Type::UINT8:
+ return 1;
+ case arrow::Type::UINT16:
+ return 2;
+ case arrow::Type::UINT32:
+ return 4;
+ case arrow::Type::UINT64:
+ return 8;
+ case arrow::Type::INT8:
+ return 1;
+ case arrow::Type::INT16:
+ return 2;
+ case arrow::Type::INT32:
+ return 4;
+ case arrow::Type::INT64:
+ return 8;
+ case arrow::Type::FLOAT:
+ return 4;
+ case arrow::Type::DOUBLE:
+ return 8;
+
+ case arrow::Type::FIXED_SIZE_BINARY:
+ return std::static_pointer_cast<DataTypeFixedString>(type)->byte_width();
+
+ case arrow::Type::STRING:
+ case arrow::Type::BINARY:
+ break;
+
+ case arrow::Type::EXTENSION: // AggregateColumn
+ break;
+
+ default:
+ break;
+ }
+
+ return 0;
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/Columns/ColumnsCommon.h b/ydb/library/arrow_clickhouse/Columns/ColumnsCommon.h
new file mode 100644
index 00000000000..5eb633d15a0
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Columns/ColumnsCommon.h
@@ -0,0 +1,120 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <Common/SipHash.h>
+#include <Common/Arena.h>
+
+#include <common/StringRef.h>
+
+/// Common helper methods for implementation of different columns.
+
+namespace CH
+{
+
+/// Counts how many bytes of `filt` are greater than zero.
+size_t countBytesInFilter(const uint8_t * filt, size_t start, size_t end);
+size_t countBytesInFilterWithNull(const uint8_t * filt, const uint8_t * null_map, size_t start, size_t end);
+
+template <typename T>
+inline StringRef serializeNumberIntoArena(T value, Arena & arena, char const *& begin)
+{
+ auto * pos = arena.allocContinue(sizeof(T), begin);
+ unalignedStore<T>(pos, value);
+ return StringRef(pos, sizeof(T));
+}
+
+template <bool fixed>
+inline StringRef serializeStringIntoArena(const StringRef & str, Arena & arena, char const *& begin)
+{
+ if constexpr (fixed)
+ {
+ auto * pos = arena.allocContinue(str.size, begin);
+ memcpy(pos, str.data, str.size);
+ return StringRef(pos, str.size);
+ }
+ else
+ {
+ StringRef res;
+ res.size = sizeof(str.size) + str.size;
+ char * pos = arena.allocContinue(res.size, begin);
+ memcpy(pos, &str.size, sizeof(str.size));
+ memcpy(pos + sizeof(str.size), str.data, str.size);
+ res.data = pos;
+ return res;
+ }
+}
+
+template <typename T>
+inline bool insertNumber(MutableColumn & column, T value)
+{
+ if constexpr (std::is_same_v<T, UInt8>)
+ return assert_cast<MutableColumnUInt8 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, UInt16>)
+ return assert_cast<MutableColumnUInt16 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, UInt32>)
+ return assert_cast<MutableColumnUInt32 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, UInt64>)
+ return assert_cast<MutableColumnUInt64 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, Int8>)
+ return assert_cast<MutableColumnInt8 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, Int16>)
+ return assert_cast<MutableColumnInt16 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, Int32>)
+ return assert_cast<MutableColumnInt32 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, Int64>)
+ return assert_cast<MutableColumnInt64 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, float>)
+ return assert_cast<MutableColumnFloat32 &>(column).Append(value).ok();
+ else if constexpr (std::is_same_v<T, double>)
+ return assert_cast<MutableColumnFloat64 &>(column).Append(value).ok();
+
+ throw Exception("unexpected type");
+}
+
+inline bool insertString(MutableColumn & column, const StringRef & value)
+{
+ return assert_cast<MutableColumnBinary &>(column).Append(arrow::util::string_view{value.data, value.size}).ok();
+}
+
+inline bool insertFixedString(MutableColumn & column, const StringRef & value)
+{
+ return assert_cast<MutableColumnFixedString &>(column).Append(arrow::util::string_view{value.data, value.size}).ok();
+}
+
+template <typename DataType>
+inline const char * deserializeNumberFromArena(arrow::NumericBuilder<DataType> & column, const char * pos)
+{
+ using T = typename arrow::TypeTraits<DataType>::CType;
+
+ T value = unalignedLoad<T>(pos);
+ column.Append(value).ok();
+ return pos + sizeof(T);
+}
+
+inline const char * deserializeStringFromArena(MutableColumnBinary & column, const char * pos)
+{
+ const size_t string_size = unalignedLoad<size_t>(pos);
+ pos += sizeof(string_size);
+
+ column.Append(pos, string_size).ok();
+ return pos + string_size;
+}
+
+inline const char * deserializeStringFromArena(MutableColumnFixedString & column, const char * pos)
+{
+ column.Append(pos).ok();
+ return pos + column.byte_width();
+}
+
+bool insertData(MutableColumn & column, const StringRef & value);
+StringRef serializeValueIntoArena(const IColumn& column, size_t row, Arena & pool, char const *& begin);
+const char * deserializeAndInsertFromArena(MutableColumn& column, const char * pos);
+void updateHashWithValue(const IColumn& column, size_t row, SipHash & hash);
+MutableColumnPtr createMutableColumn(const DataTypePtr & type);
+uint32_t fixedContiguousSize(const DataTypePtr & type);
+
+}
diff --git a/ydb/library/arrow_clickhouse/Columns/ColumnsHashing.h b/ydb/library/arrow_clickhouse/Columns/ColumnsHashing.h
new file mode 100644
index 00000000000..dd3ce650f07
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Columns/ColumnsHashing.h
@@ -0,0 +1,293 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <Common/Arena.h>
+#include <Common/PODArray.h>
+#include <Common/HashTable/HashTable.h>
+#include <Common/HashTable/HashTableKeyHolder.h>
+#include <Columns/ColumnsHashingImpl.h>
+
+#include <common/unaligned.h>
+
+#include <memory>
+#include <cassert>
+
+
+namespace CH
+{
+
+namespace ColumnsHashing
+{
+
+/// For the case when there is one numeric key.
+/// UInt8/16/32/64 for any type with corresponding bit width.
+template <typename Value, typename Mapped, typename FieldType, bool use_cache = true, bool need_offset = false>
+struct HashMethodOneNumber
+ : public columns_hashing_impl::HashMethodBase<HashMethodOneNumber<Value, Mapped, FieldType, use_cache, need_offset>, Value, Mapped, use_cache, need_offset>
+{
+ using Self = HashMethodOneNumber<Value, Mapped, FieldType, use_cache, need_offset>;
+ using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache, need_offset>;
+
+ const uint8_t * vec{};
+
+ /// If the keys of a fixed length then key_sizes contains their lengths, empty otherwise.
+ HashMethodOneNumber(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &)
+ {
+ vec = assert_cast<const ColumnUInt8 *>(key_columns[0])->raw_values();
+ }
+
+ HashMethodOneNumber(const IColumn * column)
+ {
+ vec = assert_cast<const ColumnUInt8 *>(column)->raw_values();
+ }
+
+ /// Creates context. Method is called once and result context is used in all threads.
+ using Base::createContext; /// (const HashMethodContext::Settings &) -> HashMethodContextPtr
+
+ /// Emplace key into HashTable or HashMap. If Data is HashMap, returns ptr to value, otherwise nullptr.
+ /// Data is a HashTable where to insert key from column's row.
+ /// For Serialized method, key may be placed in pool.
+ using Base::emplaceKey; /// (Data & data, size_t row, Arena & pool) -> EmplaceResult
+
+ /// Find key into HashTable or HashMap. If Data is HashMap and key was found, returns ptr to value, otherwise nullptr.
+ using Base::findKey; /// (Data & data, size_t row, Arena & pool) -> FindResult
+
+ /// Get hash value of row.
+ using Base::getHash; /// (const Data & data, size_t row, Arena & pool) -> size_t
+
+ /// Is used for default implementation in HashMethodBase.
+ FieldType getKeyHolder(size_t row, Arena &) const { return unalignedLoad<FieldType>(vec + row * sizeof(FieldType)); }
+
+ const FieldType * getKeyData() const { return reinterpret_cast<const FieldType *>(vec); }
+};
+
+
+/// For the case when there is one string key.
+template <typename Value, typename Mapped, bool place_string_to_arena = true, bool use_cache = true, bool need_offset = false>
+struct HashMethodString
+ : public columns_hashing_impl::HashMethodBase<HashMethodString<Value, Mapped, place_string_to_arena, use_cache, need_offset>, Value, Mapped, use_cache, need_offset>
+{
+ using Self = HashMethodString<Value, Mapped, place_string_to_arena, use_cache, need_offset>;
+ using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache, need_offset>;
+
+ const int * offsets{};
+ const uint8_t * chars{};
+
+ HashMethodString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &)
+ {
+ const IColumn & column = *key_columns[0];
+ const auto & column_string = assert_cast<const ColumnBinary &>(column);
+ offsets = column_string.raw_value_offsets();
+ chars = column_string.raw_data();
+ }
+
+ auto getKeyHolder(ssize_t row, [[maybe_unused]] Arena & pool) const
+ {
+ StringRef key(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1);
+
+ if constexpr (place_string_to_arena)
+ {
+ return ArenaKeyHolder{key, pool};
+ }
+ else
+ {
+ return key;
+ }
+ }
+
+protected:
+ friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
+};
+
+
+/// For the case when there is one fixed-length string key.
+template <typename Value, typename Mapped, bool place_string_to_arena = true, bool use_cache = true, bool need_offset = false>
+struct HashMethodFixedString
+ : public columns_hashing_impl::
+ HashMethodBase<HashMethodFixedString<Value, Mapped, place_string_to_arena, use_cache, need_offset>, Value, Mapped, use_cache, need_offset>
+{
+ using Self = HashMethodFixedString<Value, Mapped, place_string_to_arena, use_cache, need_offset>;
+ using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache, need_offset>;
+
+ size_t n{};
+ const uint8_t * chars{};
+
+ HashMethodFixedString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &)
+ {
+ const IColumn & column = *key_columns[0];
+ const ColumnFixedString & column_string = assert_cast<const ColumnFixedString &>(column);
+ n = column_string.byte_width();
+ chars = column_string.raw_values();
+ }
+
+ auto getKeyHolder(size_t row, [[maybe_unused]] Arena & pool) const
+ {
+ StringRef key(&chars[row * n], n);
+
+ if constexpr (place_string_to_arena)
+ {
+ return ArenaKeyHolder{key, pool};
+ }
+ else
+ {
+ return key;
+ }
+ }
+
+protected:
+ friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
+};
+
+
+/// For the case when all keys are of fixed length, and they fit in N (for example, 128) bits.
+template <
+ typename Value,
+ typename Key,
+ typename Mapped,
+ bool has_nullable_keys_ = false,
+ bool has_low_cardinality_ = false,
+ bool use_cache = true,
+ bool need_offset = false>
+struct HashMethodKeysFixed
+ : private columns_hashing_impl::BaseStateKeysFixed<Key, has_nullable_keys_>
+ , public columns_hashing_impl::HashMethodBase<HashMethodKeysFixed<Value, Key, Mapped, has_nullable_keys_, has_low_cardinality_, use_cache, need_offset>, Value, Mapped, use_cache, need_offset>
+{
+ using Self = HashMethodKeysFixed<Value, Key, Mapped, has_nullable_keys_, has_low_cardinality_, use_cache, need_offset>;
+ using BaseHashed = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache, need_offset>;
+ using Base = columns_hashing_impl::BaseStateKeysFixed<Key, has_nullable_keys_>;
+
+ static constexpr bool has_nullable_keys = has_nullable_keys_;
+
+ Sizes key_sizes;
+ size_t keys_size;
+
+#if 0 // shuffleKeyColumns disabled
+ PaddedPODArray<Key> prepared_keys;
+
+ static bool usePreparedKeys(const Sizes & key_sizes)
+ {
+ if (has_nullable_keys || sizeof(Key) > 16)
+ return false;
+
+ for (auto size : key_sizes)
+ if (size != 1 && size != 2 && size != 4 && size != 8 && size != 16)
+ return false;
+
+ return true;
+ }
+#endif
+
+ HashMethodKeysFixed(const ColumnRawPtrs & key_columns, const Sizes & key_sizes_, const HashMethodContextPtr &)
+ : Base(key_columns), key_sizes(std::move(key_sizes_)), keys_size(key_columns.size())
+ {
+#if 0
+ if (usePreparedKeys(key_sizes))
+ packFixedBatch(keys_size, Base::getActualColumns(), key_sizes, prepared_keys);
+#endif
+ }
+
+ ALWAYS_INLINE Key getKeyHolder(size_t row, Arena &) const
+ {
+ if constexpr (has_nullable_keys)
+ {
+ auto bitmap = Base::createBitmap(row);
+ return packFixed<Key>(row, keys_size, Base::getActualColumns(), key_sizes, bitmap);
+ }
+ else
+ {
+#if 0
+ if (!prepared_keys.empty())
+ return prepared_keys[row];
+#endif
+ return packFixed<Key>(row, keys_size, Base::getActualColumns(), key_sizes);
+ }
+ }
+#if 0
+ static std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> & key_columns, const Sizes & key_sizes)
+ {
+ if (!usePreparedKeys(key_sizes))
+ return {};
+
+ std::vector<IColumn *> new_columns;
+ new_columns.reserve(key_columns.size());
+
+ Sizes new_sizes;
+ auto fill_size = [&](size_t size)
+ {
+ for (size_t i = 0; i < key_sizes.size(); ++i)
+ {
+ if (key_sizes[i] == size)
+ {
+ new_columns.push_back(key_columns[i]);
+ new_sizes.push_back(size);
+ }
+ }
+ };
+
+ fill_size(16);
+ fill_size(8);
+ fill_size(4);
+ fill_size(2);
+ fill_size(1);
+
+ key_columns.swap(new_columns);
+ return new_sizes;
+ }
+#endif
+};
+
+/** Hash by concatenating serialized key values.
+ * The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts.
+ * That is, for example, for strings, it contains first the serialized length of the string, and then the bytes.
+ * Therefore, when aggregating by several strings, there is no ambiguity.
+ */
+template <typename Value, typename Mapped>
+struct HashMethodSerialized
+ : public columns_hashing_impl::HashMethodBase<HashMethodSerialized<Value, Mapped>, Value, Mapped, false>
+{
+ using Self = HashMethodSerialized<Value, Mapped>;
+ using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, false>;
+
+ ColumnRawPtrs key_columns;
+ size_t keys_size;
+
+ HashMethodSerialized(const ColumnRawPtrs & key_columns_, const Sizes & /*key_sizes*/, const HashMethodContextPtr &)
+ : key_columns(key_columns_), keys_size(key_columns_.size()) {}
+
+protected:
+ friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, false>;
+
+ ALWAYS_INLINE SerializedKeyHolder getKeyHolder(size_t row, Arena & pool) const
+ {
+ return SerializedKeyHolder{
+ serializeKeysToPoolContiguous(row, keys_size, key_columns, pool),
+ pool};
+ }
+};
+
+/// For the case when there is one string key.
+template <typename Value, typename Mapped, bool use_cache = true, bool need_offset = false>
+struct HashMethodHashed
+ : public columns_hashing_impl::HashMethodBase<HashMethodHashed<Value, Mapped, use_cache, need_offset>, Value, Mapped, use_cache, need_offset>
+{
+ using Key = UInt128;
+ using Self = HashMethodHashed<Value, Mapped, use_cache, need_offset>;
+ using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache, need_offset>;
+
+ ColumnRawPtrs key_columns;
+
+ HashMethodHashed(ColumnRawPtrs key_columns_, const Sizes &, const HashMethodContextPtr &)
+ : key_columns(std::move(key_columns_)) {}
+
+ ALWAYS_INLINE Key getKeyHolder(size_t row, Arena &) const
+ {
+ return hash128(row, key_columns.size(), key_columns);
+ }
+};
+
+}
+}
diff --git a/ydb/library/arrow_clickhouse/Columns/ColumnsHashingImpl.h b/ydb/library/arrow_clickhouse/Columns/ColumnsHashingImpl.h
new file mode 100644
index 00000000000..b3f2adf2006
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Columns/ColumnsHashingImpl.h
@@ -0,0 +1,375 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include "AggregationCommon.h"
+#include <Common/HashTable/HashTableKeyHolder.h>
+
+namespace CH
+{
+
+namespace ColumnsHashing
+{
+
+/// Generic context for HashMethod. Context is shared between multiple threads, all methods must be thread-safe.
+/// Is used for caching.
+class HashMethodContext
+{
+public:
+ virtual ~HashMethodContext() = default;
+
+ struct Settings
+ {
+ size_t max_threads;
+ };
+};
+
+using HashMethodContextPtr = std::shared_ptr<HashMethodContext>;
+
+
+namespace columns_hashing_impl
+{
+
+template <typename Value, bool consecutive_keys_optimization_>
+struct LastElementCache
+{
+ static constexpr bool consecutive_keys_optimization = consecutive_keys_optimization_;
+ Value value;
+ bool empty = true;
+ bool found = false;
+
+ bool check(const Value & value_) { return !empty && value == value_; }
+
+ template <typename Key>
+ bool check(const Key & key) { return !empty && value.first == key; }
+};
+
+template <typename Data>
+struct LastElementCache<Data, false>
+{
+ static constexpr bool consecutive_keys_optimization = false;
+};
+
+template <typename Mapped>
+class EmplaceResultImpl
+{
+ Mapped & value;
+ Mapped & cached_value;
+ bool inserted;
+
+public:
+ EmplaceResultImpl(Mapped & value_, Mapped & cached_value_, bool inserted_)
+ : value(value_), cached_value(cached_value_), inserted(inserted_) {}
+
+ bool isInserted() const { return inserted; }
+ auto & getMapped() const { return value; }
+
+ void setMapped(const Mapped & mapped)
+ {
+ cached_value = mapped;
+ value = mapped;
+ }
+};
+
+template <>
+class EmplaceResultImpl<void>
+{
+ bool inserted;
+
+public:
+ explicit EmplaceResultImpl(bool inserted_) : inserted(inserted_) {}
+ bool isInserted() const { return inserted; }
+};
+
+/// FindResult optionally may contain pointer to value and offset in hashtable buffer.
+/// Only bool found is required.
+/// So we will have 4 different specializations for FindResultImpl
+class FindResultImplBase
+{
+ bool found;
+
+public:
+ explicit FindResultImplBase(bool found_) : found(found_) {}
+ bool isFound() const { return found; }
+};
+
+template <bool need_offset = false>
+class FindResultImplOffsetBase
+{
+public:
+ constexpr static bool has_offset = need_offset;
+ explicit FindResultImplOffsetBase(size_t /* off */) {}
+};
+
+template <>
+class FindResultImplOffsetBase<true>
+{
+ size_t offset;
+public:
+ constexpr static bool has_offset = true;
+
+ explicit FindResultImplOffsetBase(size_t off) : offset(off) {}
+ ALWAYS_INLINE size_t getOffset() const { return offset; }
+};
+
+template <typename Mapped, bool need_offset = false>
+class FindResultImpl : public FindResultImplBase, public FindResultImplOffsetBase<need_offset>
+{
+ Mapped * value;
+
+public:
+ FindResultImpl()
+ : FindResultImplBase(false), FindResultImplOffsetBase<need_offset>(0)
+ {}
+
+ FindResultImpl(Mapped * value_, bool found_, size_t off)
+ : FindResultImplBase(found_), FindResultImplOffsetBase<need_offset>(off), value(value_) {}
+ Mapped & getMapped() const { return *value; }
+};
+
+template <bool need_offset>
+class FindResultImpl<void, need_offset> : public FindResultImplBase, public FindResultImplOffsetBase<need_offset>
+{
+public:
+ FindResultImpl(bool found_, size_t off) : FindResultImplBase(found_), FindResultImplOffsetBase<need_offset>(off) {}
+};
+
+template <typename Derived, typename Value, typename Mapped, bool consecutive_keys_optimization, bool need_offset = false>
+class HashMethodBase
+{
+public:
+ using EmplaceResult = EmplaceResultImpl<Mapped>;
+ using FindResult = FindResultImpl<Mapped, need_offset>;
+ static constexpr bool has_mapped = !std::is_same<Mapped, void>::value;
+ using Cache = LastElementCache<Value, consecutive_keys_optimization>;
+
+ static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; }
+
+ template <typename Data>
+ ALWAYS_INLINE EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool)
+ {
+ auto key_holder = static_cast<Derived &>(*this).getKeyHolder(row, pool);
+ return emplaceImpl(key_holder, data);
+ }
+
+ template <typename Data>
+ ALWAYS_INLINE FindResult findKey(Data & data, size_t row, Arena & pool)
+ {
+ auto key_holder = static_cast<Derived &>(*this).getKeyHolder(row, pool);
+ return findKeyImpl(keyHolderGetKey(key_holder), data);
+ }
+
+ template <typename Data>
+ ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & pool)
+ {
+ auto key_holder = static_cast<Derived &>(*this).getKeyHolder(row, pool);
+ return data.hash(keyHolderGetKey(key_holder));
+ }
+
+protected:
+ Cache cache;
+
+ HashMethodBase()
+ {
+ if constexpr (consecutive_keys_optimization)
+ {
+ if constexpr (has_mapped)
+ {
+ /// Init PairNoInit elements.
+ cache.value.second = Mapped();
+ cache.value.first = {};
+ }
+ else
+ cache.value = Value();
+ }
+ }
+
+ template <typename Data, typename KeyHolder>
+ ALWAYS_INLINE EmplaceResult emplaceImpl(KeyHolder & key_holder, Data & data)
+ {
+ if constexpr (Cache::consecutive_keys_optimization)
+ {
+ if (cache.found && cache.check(keyHolderGetKey(key_holder)))
+ {
+ if constexpr (has_mapped)
+ return EmplaceResult(cache.value.second, cache.value.second, false);
+ else
+ return EmplaceResult(false);
+ }
+ }
+
+ typename Data::LookupResult it;
+ bool inserted = false;
+ data.emplace(key_holder, it, inserted);
+
+ [[maybe_unused]] Mapped * cached = nullptr;
+ if constexpr (has_mapped)
+ cached = &it->getMapped();
+
+ if (inserted)
+ {
+ if constexpr (has_mapped)
+ {
+ new (&it->getMapped()) Mapped();
+ }
+ }
+
+ if constexpr (consecutive_keys_optimization)
+ {
+ cache.found = true;
+ cache.empty = false;
+
+ if constexpr (has_mapped)
+ {
+ cache.value.first = it->getKey();
+ cache.value.second = it->getMapped();
+ cached = &cache.value.second;
+ }
+ else
+ {
+ cache.value = it->getKey();
+ }
+ }
+
+ if constexpr (has_mapped)
+ return EmplaceResult(it->getMapped(), *cached, inserted);
+ else
+ return EmplaceResult(inserted);
+ }
+
+ template <typename Data, typename Key>
+ ALWAYS_INLINE FindResult findKeyImpl(Key key, Data & data)
+ {
+ if constexpr (Cache::consecutive_keys_optimization)
+ {
+ /// It's possible to support such combination, but code will became more complex.
+ /// Now there's not place where we need this options enabled together
+ static_assert(!FindResult::has_offset, "`consecutive_keys_optimization` and `has_offset` are conflicting options");
+ if (cache.check(key))
+ {
+ if constexpr (has_mapped)
+ return FindResult(&cache.value.second, cache.found, 0);
+ else
+ return FindResult(cache.found, 0);
+ }
+ }
+
+ auto it = data.find(key);
+
+ if constexpr (consecutive_keys_optimization)
+ {
+ cache.found = it != nullptr;
+ cache.empty = false;
+
+ if constexpr (has_mapped)
+ {
+ cache.value.first = key;
+ if (it)
+ {
+ cache.value.second = it->getMapped();
+ }
+ }
+ else
+ {
+ cache.value = key;
+ }
+ }
+
+ size_t offset = 0;
+ if constexpr (FindResult::has_offset)
+ {
+ offset = it ? data.offsetInternal(it) : 0;
+ }
+ if constexpr (has_mapped)
+ return FindResult(it ? &it->getMapped() : nullptr, it != nullptr, offset);
+ else
+ return FindResult(it != nullptr, offset);
+ }
+};
+
+
+template <typename T>
+struct MappedCache : public PaddedPODArray<T> {};
+
+template <>
+struct MappedCache<void> {};
+
+
+/// This class is designed to provide the functionality that is required for
+/// supporting nullable keys in HashMethodKeysFixed. If there are
+/// no nullable keys, this class is merely implemented as an empty shell.
+template <typename Key, bool has_nullable_keys>
+class BaseStateKeysFixed;
+
+/// Case where nullable keys are supported.
+template <typename Key>
+class BaseStateKeysFixed<Key, true>
+{
+protected:
+ BaseStateKeysFixed(const ColumnRawPtrs & key_columns)
+ {
+ actual_columns.reserve(key_columns.size());
+ for (const auto & col : key_columns)
+ {
+ actual_columns.push_back(col);
+ }
+ }
+
+ /// Return the columns which actually contain the values of the keys.
+ /// For a given key column, if it is nullable, we return its nested
+ /// column. Otherwise we return the key column itself.
+ inline const ColumnRawPtrs & getActualColumns() const
+ {
+ return actual_columns;
+ }
+
+ /// Create a bitmap that indicates whether, for a particular row,
+ /// a key column bears a null value or not.
+ KeysNullMap<Key> createBitmap(size_t row) const
+ {
+ KeysNullMap<Key> bitmap{};
+
+ for (size_t k = 0; k < actual_columns.size(); ++k)
+ {
+ bool is_null = actual_columns[k]->IsNull(row);
+ if (is_null)
+ {
+ size_t bucket = k / 8;
+ size_t offset = k % 8;
+ bitmap[bucket] |= UInt8(1) << offset;
+ }
+ }
+
+ return bitmap;
+ }
+
+private:
+ ColumnRawPtrs actual_columns;
+};
+
+/// Case where nullable keys are not supported.
+template <typename Key>
+class BaseStateKeysFixed<Key, false>
+{
+protected:
+ BaseStateKeysFixed(const ColumnRawPtrs & columns) : actual_columns(columns) {}
+
+ const ColumnRawPtrs & getActualColumns() const { return actual_columns; }
+
+ KeysNullMap<Key> createBitmap(size_t) const
+ {
+ throw Exception{"Internal error: calling createBitmap() for non-nullable keys is forbidden"};
+ }
+
+private:
+ ColumnRawPtrs actual_columns;
+};
+
+}
+
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/Allocator.cpp b/ydb/library/arrow_clickhouse/Common/Allocator.cpp
new file mode 100644
index 00000000000..96a2ae6ad0a
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/Allocator.cpp
@@ -0,0 +1,35 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#include <Common/Allocator.h>
+
+namespace CH
+{
+
+/** Keep definition of this constant in cpp file; otherwise its value
+ * is inlined into allocator code making it impossible to override it
+ * in third-party code.
+ *
+ * Note: extern may seem redundant, but is actually needed due to bug in GCC.
+ * See also: https://gcc.gnu.org/legacy-ml/gcc-help/2017-12/msg00021.html
+ */
+#ifdef NDEBUG
+ __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 64 * (1ULL << 20);
+#else
+ /**
+ * In debug build, use small mmap threshold to reproduce more memory
+ * stomping bugs. Along with ASLR it will hopefully detect more issues than
+ * ASan. The program may fail due to the limit on number of memory mappings.
+ *
+ * Not too small to avoid too quick exhaust of memory mappings.
+ */
+ __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 16384;
+#endif
+
+template class Allocator<false, false>;
+template class Allocator<true, false>;
+template class Allocator<false, true>;
+template class Allocator<true, true>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/Allocator.h b/ydb/library/arrow_clickhouse/Common/Allocator.h
new file mode 100644
index 00000000000..b9c1aa1247c
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/Allocator.h
@@ -0,0 +1,344 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <string.h>
+
+#ifdef NDEBUG
+ #define ALLOCATOR_ASLR 0
+#else
+ #define ALLOCATOR_ASLR 1
+#endif
+
+#if !defined(__APPLE__) && !defined(__FreeBSD__)
+#include <malloc.h>
+#endif
+
+#include <cstdlib>
+#include <algorithm>
+#include <sys/mman.h>
+
+
+#include <common/mremap.h>
+#include <common/getPageSize.h>
+
+#include <Common/Allocator_fwd.h>
+
+/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS
+#ifndef MAP_ANONYMOUS
+#define MAP_ANONYMOUS MAP_ANON
+#endif
+
+namespace CH
+{
+
+/**
+ * Many modern allocators (for example, tcmalloc) do not do a mremap for
+ * realloc, even in case of large enough chunks of memory. Although this allows
+ * you to increase performance and reduce memory consumption during realloc.
+ * To fix this, we do mremap manually if the chunk of memory is large enough.
+ * The threshold (64 MB) is chosen quite large, since changing the address
+ * space is very slow, especially in the case of a large number of threads. We
+ * expect that the set of operations mmap/something to do/mremap can only be
+ * performed about 1000 times per second.
+ *
+ * P.S. This is also required, because tcmalloc can not allocate a chunk of
+ * memory greater than 16 GB.
+ *
+ * P.P.S. Note that MMAP_THRESHOLD symbol is intentionally made weak. It allows
+ * to override it during linkage when using ClickHouse as a library in
+ * third-party applications which may already use own allocator doing mmaps
+ * in the implementation of alloc/realloc.
+ */
+extern const size_t MMAP_THRESHOLD;
+
+static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
+
+
+/** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena.
+ * Also used in hash tables.
+ * The interface is different from std::allocator
+ * - the presence of the method realloc, which for large chunks of memory uses mremap;
+ * - passing the size into the `free` method;
+ * - by the presence of the `alignment` argument;
+ * - the possibility of zeroing memory (used in hash tables);
+ * - random hint address for mmap
+ * - mmap_threshold for using mmap less or more
+ */
+template <bool clear_memory_, bool mmap_populate>
+class Allocator
+{
+public:
+ /// Allocate memory range.
+ void * alloc(size_t size, size_t alignment = 0)
+ {
+ checkSize(size);
+ return allocNoTrack(size, alignment);
+ }
+
+ /// Free memory range.
+ void free(void * buf, size_t size)
+ {
+ try
+ {
+ checkSize(size);
+ freeNoTrack(buf, size);
+ }
+ catch (...)
+ {
+ //DB::tryLogCurrentException("Allocator::free");
+ throw;
+ }
+ }
+
+ /** Enlarge memory range.
+ * Data from old range is moved to the beginning of new range.
+ * Address of memory range could change.
+ */
+ void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0)
+ {
+ checkSize(new_size);
+
+ if (old_size == new_size)
+ {
+ /// nothing to do.
+ /// BTW, it's not possible to change alignment while doing realloc.
+ }
+ else if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD
+ && alignment <= MALLOC_MIN_ALIGNMENT)
+ {
+ void * new_buf = ::realloc(buf, new_size);
+ if (nullptr == new_buf)
+ throw std::runtime_error("Allocator: Cannot realloc");
+
+ buf = new_buf;
+ if constexpr (clear_memory)
+ if (new_size > old_size)
+ memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
+ }
+ else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD)
+ {
+ // On apple and freebsd self-implemented mremap used (common/mremap.h)
+ buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE,
+ PROT_READ | PROT_WRITE, mmap_flags, -1, 0);
+ if (MAP_FAILED == buf)
+ throw std::runtime_error("Allocator: Cannot mremap memory");
+
+ /// No need for zero-fill, because mmap guarantees it.
+ }
+ else if (new_size < MMAP_THRESHOLD)
+ {
+ void * new_buf = allocNoTrack(new_size, alignment);
+ memcpy(new_buf, buf, std::min(old_size, new_size));
+ freeNoTrack(buf, old_size);
+ buf = new_buf;
+ }
+ else
+ {
+ /// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods.
+
+ void * new_buf = alloc(new_size, alignment);
+ memcpy(new_buf, buf, std::min(old_size, new_size));
+ free(buf, old_size);
+ buf = new_buf;
+ }
+
+ return buf;
+ }
+
+protected:
+ static constexpr size_t getStackThreshold()
+ {
+ return 0;
+ }
+
+ static constexpr bool clear_memory = clear_memory_;
+
+ // Freshly mmapped pages are copy-on-write references to a global zero page.
+ // On the first write, a page fault occurs, and an actual writable page is
+ // allocated. If we are going to use this memory soon, such as when resizing
+ // hash tables, it makes sense to pre-fault the pages by passing
+ // MAP_POPULATE to mmap(). This takes some time, but should be faster
+ // overall than having a hot loop interrupted by page faults.
+ // It is only supported on Linux.
+ static constexpr int mmap_flags = MAP_PRIVATE | MAP_ANONYMOUS
+#if defined(OS_LINUX)
+ | (mmap_populate ? MAP_POPULATE : 0)
+#endif
+ ;
+
+private:
+ void * allocNoTrack(size_t size, size_t alignment)
+ {
+ void * buf;
+ size_t mmap_min_alignment = ::getPageSize();
+
+ if (size >= MMAP_THRESHOLD)
+ {
+ if (alignment > mmap_min_alignment)
+ throw std::runtime_error("Too large alignment: more than page size when allocating");
+
+ buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE,
+ mmap_flags, -1, 0);
+ if (MAP_FAILED == buf)
+ throw std::runtime_error("Allocator: Cannot mmap");
+
+ /// No need for zero-fill, because mmap guarantees it.
+ }
+ else
+ {
+ if (alignment <= MALLOC_MIN_ALIGNMENT)
+ {
+ if constexpr (clear_memory)
+ buf = ::calloc(size, 1);
+ else
+ buf = ::malloc(size);
+
+ if (nullptr == buf)
+ throw std::runtime_error("Allocator: Cannot malloc");
+ }
+ else
+ {
+ buf = nullptr;
+ int res = posix_memalign(&buf, alignment, size);
+
+ if (0 != res)
+ throw std::runtime_error("Cannot allocate memory (posix_memalign)");
+
+ if constexpr (clear_memory)
+ memset(buf, 0, size);
+ }
+ }
+ return buf;
+ }
+
+ void freeNoTrack(void * buf, size_t size)
+ {
+ if (size >= MMAP_THRESHOLD)
+ {
+ if (0 != munmap(buf, size))
+ throw std::runtime_error("Allocator: Cannot munmap");
+ }
+ else
+ {
+ ::free(buf);
+ }
+ }
+
+ void checkSize(size_t size)
+ {
+ /// More obvious exception in case of possible overflow (instead of just "Cannot mmap").
+ if (size >= 0x8000000000000000ULL)
+ throw std::runtime_error("Too large size passed to allocator. It indicates an error.");
+ }
+
+#ifndef NDEBUG
+ /// In debug builds, request mmap() at random addresses (a kind of ASLR), to
+ /// reproduce more memory stomping bugs. Note that Linux doesn't do it by
+ /// default. This may lead to worse TLB performance.
+ void * getMmapHint()
+ {
+ //return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(thread_local_rng));
+ return nullptr;
+ }
+#else
+ void * getMmapHint()
+ {
+ return nullptr;
+ }
+#endif
+};
+
+/** When using AllocatorWithStackMemory, located on the stack,
+ * GCC 4.9 mistakenly assumes that we can call `free` from a pointer to the stack.
+ * In fact, the combination of conditions inside AllocatorWithStackMemory does not allow this.
+ */
+#if !defined(__clang__)
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wfree-nonheap-object"
+#endif
+
+/** Allocator with optimization to place small memory ranges in automatic memory.
+ */
+template <typename Base, size_t _initial_bytes, size_t Alignment>
+class AllocatorWithStackMemory : private Base
+{
+private:
+ alignas(Alignment) char stack_memory[_initial_bytes];
+
+public:
+ static constexpr size_t initial_bytes = _initial_bytes;
+
+ /// Do not use boost::noncopyable to avoid the warning about direct base
+ /// being inaccessible due to ambiguity, when derived classes are also
+ /// noncopiable (-Winaccessible-base).
+ AllocatorWithStackMemory(const AllocatorWithStackMemory&) = delete;
+ AllocatorWithStackMemory & operator = (const AllocatorWithStackMemory&) = delete;
+ AllocatorWithStackMemory() = default;
+ ~AllocatorWithStackMemory() = default;
+
+ void * alloc(size_t size)
+ {
+ if (size <= initial_bytes)
+ {
+ if constexpr (Base::clear_memory)
+ memset(stack_memory, 0, initial_bytes);
+ return stack_memory;
+ }
+
+ return Base::alloc(size, Alignment);
+ }
+
+ void free(void * buf, size_t size)
+ {
+ if (size > initial_bytes)
+ Base::free(buf, size);
+ }
+
+ void * realloc(void * buf, size_t old_size, size_t new_size)
+ {
+ /// Was in stack_memory, will remain there.
+ if (new_size <= initial_bytes)
+ return buf;
+
+ /// Already was big enough to not fit in stack_memory.
+ if (old_size > initial_bytes)
+ return Base::realloc(buf, old_size, new_size, Alignment);
+
+ /// Was in stack memory, but now will not fit there.
+ void * new_buf = Base::alloc(new_size, Alignment);
+ memcpy(new_buf, buf, old_size);
+ return new_buf;
+ }
+
+protected:
+ static constexpr size_t getStackThreshold()
+ {
+ return initial_bytes;
+ }
+};
+
+// A constant that gives the number of initially available bytes in
+// the allocator. Used to check that this number is in sync with the
+// initial size of array or hash table that uses the allocator.
+template<typename TAllocator>
+constexpr size_t allocatorInitialBytes = 0;
+
+template<typename Base, size_t initial_bytes, size_t Alignment>
+constexpr size_t allocatorInitialBytes<AllocatorWithStackMemory<
+ Base, initial_bytes, Alignment>> = initial_bytes;
+
+/// Prevent implicit template instantiation of Allocator
+
+extern template class Allocator<false, false>;
+extern template class Allocator<true, false>;
+extern template class Allocator<false, true>;
+extern template class Allocator<true, true>;
+
+}
+
+#if !defined(__clang__)
+#pragma GCC diagnostic pop
+#endif
diff --git a/ydb/library/arrow_clickhouse/Common/Allocator_fwd.h b/ydb/library/arrow_clickhouse/Common/Allocator_fwd.h
new file mode 100644
index 00000000000..6e1b6b62257
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/Allocator_fwd.h
@@ -0,0 +1,19 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+/**
+ * This file provides forward declarations for Allocator.
+ */
+
+namespace CH
+{
+
+template <bool clear_memory_, bool mmap_populate = false>
+class Allocator;
+
+template <typename Base, size_t N = 64, size_t Alignment = 1>
+class AllocatorWithStackMemory;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/Arena.h b/ydb/library/arrow_clickhouse/Common/Arena.h
new file mode 100644
index 00000000000..7f86e70e5ae
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/Arena.h
@@ -0,0 +1,311 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <string.h>
+#include <memory>
+#include <vector>
+
+#include <Common/memcpySmall.h>
+#include <Common/Allocator.h>
+
+namespace CH
+{
+
+/** Memory pool to append something. For example, short strings.
+ * Usage scenario:
+ * - put lot of strings inside pool, keep their addresses;
+ * - addresses remain valid during lifetime of pool;
+ * - at destruction of pool, all memory is freed;
+ * - memory is allocated and freed by large MemoryChunks;
+ * - freeing parts of data is not possible (but look at ArenaWithFreeLists if you need);
+ */
+class Arena //: private boost::noncopyable
+{
+private:
+ /// Padding allows to use 'memcpySmallAllowReadWriteOverflow15' instead of 'memcpy'.
+ static constexpr size_t pad_right = 15;
+
+ /// Contiguous MemoryChunk of memory and pointer to free space inside it. Member of single-linked list.
+ struct alignas(16) MemoryChunk : private Allocator<false> /// empty base optimization
+ {
+ char * begin;
+ char * pos;
+ char * end; /// does not include padding.
+
+ MemoryChunk * prev;
+
+ MemoryChunk(size_t size_, MemoryChunk * prev_)
+ {
+ begin = reinterpret_cast<char *>(Allocator<false>::alloc(size_));
+ pos = begin;
+ end = begin + size_ - pad_right;
+ prev = prev_;
+
+ //ASAN_POISON_MEMORY_REGION(begin, size_);
+ }
+
+ ~MemoryChunk()
+ {
+ /// We must unpoison the memory before returning to the allocator,
+ /// because the allocator might not have asan integration, and the
+ /// memory would stay poisoned forever. If the allocator supports
+ /// asan, it will correctly poison the memory by itself.
+ //ASAN_UNPOISON_MEMORY_REGION(begin, size());
+
+ Allocator<false>::free(begin, size());
+
+ if (prev)
+ delete prev;
+ }
+
+ size_t size() const { return end + pad_right - begin; }
+ size_t remaining() const { return end - pos; }
+ };
+
+ size_t growth_factor;
+ size_t linear_growth_threshold;
+
+ /// Last contiguous MemoryChunk of memory.
+ MemoryChunk * head;
+ size_t size_in_bytes;
+ size_t page_size;
+
+ static size_t roundUpToPageSize(size_t s, size_t page_size)
+ {
+ return (s + page_size - 1) / page_size * page_size;
+ }
+
+ /// If MemoryChunks size is less than 'linear_growth_threshold', then use exponential growth, otherwise - linear growth
+ /// (to not allocate too much excessive memory).
+ size_t nextSize(size_t min_next_size) const
+ {
+ size_t size_after_grow = 0;
+
+ if (head->size() < linear_growth_threshold)
+ {
+ size_after_grow = std::max(min_next_size, head->size() * growth_factor);
+ }
+ else
+ {
+ // allocContinue() combined with linear growth results in quadratic
+ // behavior: we append the data by small amounts, and when it
+ // doesn't fit, we create a new MemoryChunk and copy all the previous data
+ // into it. The number of times we do this is directly proportional
+ // to the total size of data that is going to be serialized. To make
+ // the copying happen less often, round the next size up to the
+ // linear_growth_threshold.
+ size_after_grow = ((min_next_size + linear_growth_threshold - 1)
+ / linear_growth_threshold) * linear_growth_threshold;
+ }
+
+ assert(size_after_grow >= min_next_size);
+ return roundUpToPageSize(size_after_grow, page_size);
+ }
+
+ /// Add next contiguous MemoryChunk of memory with size not less than specified.
+ void NO_INLINE addMemoryChunk(size_t min_size)
+ {
+ head = new MemoryChunk(nextSize(min_size + pad_right), head);
+ size_in_bytes += head->size();
+ }
+
+ friend class ArenaAllocator;
+ template <size_t> friend class AlignedArenaAllocator;
+
+public:
+ explicit Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2, size_t linear_growth_threshold_ = 128 * 1024 * 1024)
+ : growth_factor(growth_factor_), linear_growth_threshold(linear_growth_threshold_),
+ head(new MemoryChunk(initial_size_, nullptr)), size_in_bytes(head->size()),
+ page_size(static_cast<size_t>(::getPageSize()))
+ {
+ }
+
+ ~Arena()
+ {
+ delete head;
+ }
+
+ /// Get piece of memory, without alignment.
+ char * alloc(size_t size)
+ {
+ if (unlikely(head->pos + size > head->end))
+ addMemoryChunk(size);
+
+ char * res = head->pos;
+ head->pos += size;
+ //ASAN_UNPOISON_MEMORY_REGION(res, size + pad_right);
+ return res;
+ }
+
+ /// Get piece of memory with alignment
+ char * alignedAlloc(size_t size, size_t alignment)
+ {
+ do
+ {
+ void * head_pos = head->pos;
+ size_t space = head->end - head->pos;
+
+ auto * res = static_cast<char *>(std::align(alignment, size, head_pos, space));
+ if (res)
+ {
+ head->pos = static_cast<char *>(head_pos);
+ head->pos += size;
+ //ASAN_UNPOISON_MEMORY_REGION(res, size + pad_right);
+ return res;
+ }
+
+ addMemoryChunk(size + alignment);
+ } while (true);
+ }
+
+ template <typename T>
+ T * alloc()
+ {
+ return reinterpret_cast<T *>(alignedAlloc(sizeof(T), alignof(T)));
+ }
+
+ /** Rollback just performed allocation.
+ * Must pass size not more that was just allocated.
+ * Return the resulting head pointer, so that the caller can assert that
+ * the allocation it intended to roll back was indeed the last one.
+ */
+ void * rollback(size_t size)
+ {
+ head->pos -= size;
+ //ASAN_POISON_MEMORY_REGION(head->pos, size + pad_right);
+ return head->pos;
+ }
+
+ /** Begin or expand a contiguous range of memory.
+ * 'range_start' is the start of range. If nullptr, a new range is
+ * allocated.
+ * If there is no space in the current MemoryChunk to expand the range,
+ * the entire range is copied to a new, bigger memory MemoryChunk, and the value
+ * of 'range_start' is updated.
+ * If the optional 'start_alignment' is specified, the start of range is
+ * kept aligned to this value.
+ *
+ * NOTE This method is usable only for the last allocation made on this
+ * Arena. For earlier allocations, see 'realloc' method.
+ */
+ char * allocContinue(size_t additional_bytes, char const *& range_start,
+ size_t start_alignment = 0)
+ {
+ /*
+ * Allocating zero bytes doesn't make much sense. Also, a zero-sized
+ * range might break the invariant that the range begins at least before
+ * the current MemoryChunk end.
+ */
+ assert(additional_bytes > 0);
+
+ if (!range_start)
+ {
+ // Start a new memory range.
+ char * result = start_alignment
+ ? alignedAlloc(additional_bytes, start_alignment)
+ : alloc(additional_bytes);
+
+ range_start = result;
+ return result;
+ }
+
+ // Extend an existing memory range with 'additional_bytes'.
+
+ // This method only works for extending the last allocation. For lack of
+ // original size, check a weaker condition: that 'begin' is at least in
+ // the current MemoryChunk.
+ assert(range_start >= head->begin);
+ assert(range_start < head->end);
+
+ if (head->pos + additional_bytes <= head->end)
+ {
+ // The new size fits into the last MemoryChunk, so just alloc the
+ // additional size. We can alloc without alignment here, because it
+ // only applies to the start of the range, and we don't change it.
+ return alloc(additional_bytes);
+ }
+
+ // New range doesn't fit into this MemoryChunk, will copy to a new one.
+ //
+ // Note: among other things, this method is used to provide a hack-ish
+ // implementation of realloc over Arenas in ArenaAllocators. It wastes a
+ // lot of memory -- quadratically so when we reach the linear allocation
+ // threshold. This deficiency is intentionally left as is, and should be
+ // solved not by complicating this method, but by rethinking the
+ // approach to memory management for aggregate function states, so that
+ // we can provide a proper realloc().
+ const size_t existing_bytes = head->pos - range_start;
+ const size_t new_bytes = existing_bytes + additional_bytes;
+ const char * old_range = range_start;
+
+ char * new_range = start_alignment
+ ? alignedAlloc(new_bytes, start_alignment)
+ : alloc(new_bytes);
+
+ memcpy(new_range, old_range, existing_bytes);
+
+ range_start = new_range;
+ return new_range + existing_bytes;
+ }
+
+ /// NOTE Old memory region is wasted.
+ char * realloc(const char * old_data, size_t old_size, size_t new_size)
+ {
+ char * res = alloc(new_size);
+ if (old_data)
+ {
+ memcpy(res, old_data, old_size);
+ //ASAN_POISON_MEMORY_REGION(old_data, old_size);
+ }
+ return res;
+ }
+
+ char * alignedRealloc(const char * old_data, size_t old_size, size_t new_size, size_t alignment)
+ {
+ char * res = alignedAlloc(new_size, alignment);
+ if (old_data)
+ {
+ memcpy(res, old_data, old_size);
+ //ASAN_POISON_MEMORY_REGION(old_data, old_size);
+ }
+ return res;
+ }
+
+ /// Insert string without alignment.
+ const char * insert(const char * data, size_t size)
+ {
+ char * res = alloc(size);
+ memcpy(res, data, size);
+ return res;
+ }
+
+ const char * alignedInsert(const char * data, size_t size, size_t alignment)
+ {
+ char * res = alignedAlloc(size, alignment);
+ memcpy(res, data, size);
+ return res;
+ }
+
+ /// Size of MemoryChunks in bytes.
+ size_t size() const
+ {
+ return size_in_bytes;
+ }
+
+ /// Bad method, don't use it -- the MemoryChunks are not your business, the entire
+ /// purpose of the arena code is to manage them for you, so if you find
+ /// yourself having to use this method, probably you're doing something wrong.
+ size_t remainingSpaceInCurrentMemoryChunk() const
+ {
+ return head->remaining();
+ }
+};
+
+using ArenaPtr = std::shared_ptr<Arena>;
+using Arenas = std::vector<ArenaPtr>;
+
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/CMakeLists.txt b/ydb/library/arrow_clickhouse/Common/CMakeLists.txt
new file mode 100644
index 00000000000..a455b526c0b
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/CMakeLists.txt
@@ -0,0 +1,23 @@
+
+# This file was gererated by the build system used internally in the Yandex monorepo.
+# Only simple modifications are allowed (adding source-files to targets, adding simple properties
+# like target_include_directories). These modifications will be ported to original
+# ya.make files by maintainers. Any complex modifications which can't be ported back to the
+# original buildsystem will not be accepted.
+
+
+
+add_library(library-arrow_clickhouse-Common)
+target_include_directories(library-arrow_clickhouse-Common PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/yql/udfs/common/clickhouse/client/base
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse
+)
+target_link_libraries(library-arrow_clickhouse-Common PUBLIC
+ contrib-libs-cxxsupp
+ yutil
+ libs-apache-arrow
+)
+target_sources(library-arrow_clickhouse-Common PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/Common/Allocator.cpp
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/Common/PODArray.cpp
+)
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/FixedHashMap.h b/ydb/library/arrow_clickhouse/Common/HashTable/FixedHashMap.h
new file mode 100644
index 00000000000..7358fc0ea82
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/FixedHashMap.h
@@ -0,0 +1,186 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <Common/HashTable/FixedHashTable.h>
+#include <Common/HashTable/HashMap.h>
+
+namespace CH
+{
+
+template <typename Key, typename TMapped, typename TState = HashTableNoState>
+struct FixedHashMapCell
+{
+ using Mapped = TMapped;
+ using State = TState;
+
+ using value_type = PairNoInit<Key, Mapped>;
+ using mapped_type = TMapped;
+
+ bool full;
+ Mapped mapped;
+
+ FixedHashMapCell() {} //-V730
+ FixedHashMapCell(const Key &, const State &) : full(true) {}
+ FixedHashMapCell(const value_type & value_, const State &) : full(true), mapped(value_.second) {}
+
+ const VoidKey getKey() const { return {}; }
+ Mapped & getMapped() { return mapped; }
+ const Mapped & getMapped() const { return mapped; }
+
+ bool isZero(const State &) const { return !full; }
+ void setZero() { full = false; }
+
+ /// Similar to FixedHashSetCell except that we need to contain a pointer to the Mapped field.
+ /// Note that we have to assemble a continuous layout for the value_type on each call of getValue().
+ struct CellExt
+ {
+ CellExt() {} //-V730
+ CellExt(Key && key_, const FixedHashMapCell * ptr_) : key(key_), ptr(const_cast<FixedHashMapCell *>(ptr_)) {}
+ void update(Key && key_, const FixedHashMapCell * ptr_)
+ {
+ key = key_;
+ ptr = const_cast<FixedHashMapCell *>(ptr_);
+ }
+ Key key;
+ FixedHashMapCell * ptr;
+
+ const Key & getKey() const { return key; }
+ Mapped & getMapped() { return ptr->mapped; }
+ const Mapped & getMapped() const { return ptr->mapped; }
+ const value_type getValue() const { return {key, ptr->mapped}; }
+ };
+};
+
+
+/// In case when we can encode empty cells with zero mapped values.
+template <typename Key, typename TMapped, typename TState = HashTableNoState>
+struct FixedHashMapImplicitZeroCell
+{
+ using Mapped = TMapped;
+ using State = TState;
+
+ using value_type = PairNoInit<Key, Mapped>;
+ using mapped_type = TMapped;
+
+ Mapped mapped;
+
+ FixedHashMapImplicitZeroCell() {}
+ FixedHashMapImplicitZeroCell(const Key &, const State &) {}
+ FixedHashMapImplicitZeroCell(const value_type & value_, const State &) : mapped(value_.second) {}
+
+ const VoidKey getKey() const { return {}; }
+ Mapped & getMapped() { return mapped; }
+ const Mapped & getMapped() const { return mapped; }
+
+ bool isZero(const State &) const { return !mapped; }
+ void setZero() { mapped = {}; }
+
+ /// Similar to FixedHashSetCell except that we need to contain a pointer to the Mapped field.
+ /// Note that we have to assemble a continuous layout for the value_type on each call of getValue().
+ struct CellExt
+ {
+ CellExt() {} //-V730
+ CellExt(Key && key_, const FixedHashMapImplicitZeroCell * ptr_) : key(key_), ptr(const_cast<FixedHashMapImplicitZeroCell *>(ptr_)) {}
+ void update(Key && key_, const FixedHashMapImplicitZeroCell * ptr_)
+ {
+ key = key_;
+ ptr = const_cast<FixedHashMapImplicitZeroCell *>(ptr_);
+ }
+ Key key;
+ FixedHashMapImplicitZeroCell * ptr;
+
+ const Key & getKey() const { return key; }
+ Mapped & getMapped() { return ptr->mapped; }
+ const Mapped & getMapped() const { return ptr->mapped; }
+ const value_type getValue() const { return {key, ptr->mapped}; }
+ };
+};
+
+
+template <
+ typename Key,
+ typename Mapped,
+ typename Cell = FixedHashMapCell<Key, Mapped>,
+ typename Size = FixedHashTableStoredSize<Cell>,
+ typename Allocator = HashTableAllocator>
+class FixedHashMap : public FixedHashTable<Key, Cell, Size, Allocator>
+{
+public:
+ using Base = FixedHashTable<Key, Cell, Size, Allocator>;
+ using Self = FixedHashMap;
+ using LookupResult = typename Base::LookupResult;
+
+ using Base::Base;
+
+ template <typename Func>
+ void ALWAYS_INLINE mergeToViaEmplace(Self & that, Func && func)
+ {
+ for (auto it = this->begin(), end = this->end(); it != end; ++it)
+ {
+ typename Self::LookupResult res_it;
+ bool inserted;
+ that.emplace(it->getKey(), res_it, inserted, it.getHash());
+ func(res_it->getMapped(), it->getMapped(), inserted);
+ }
+ }
+
+ template <typename Func>
+ void ALWAYS_INLINE mergeToViaFind(Self & that, Func && func)
+ {
+ for (auto it = this->begin(), end = this->end(); it != end; ++it)
+ {
+ auto res_it = that.find(it->getKey(), it.getHash());
+ if (!res_it)
+ func(it->getMapped(), it->getMapped(), false);
+ else
+ func(res_it->getMapped(), it->getMapped(), true);
+ }
+ }
+
+ template <typename Func>
+ void forEachValue(Func && func)
+ {
+ for (auto & v : *this)
+ func(v.getKey(), v.getMapped());
+ }
+
+ template <typename Func>
+ void forEachMapped(Func && func)
+ {
+ for (auto & v : *this)
+ func(v.getMapped());
+ }
+
+ Mapped & ALWAYS_INLINE operator[](const Key & x)
+ {
+ LookupResult it;
+ bool inserted;
+ this->emplace(x, it, inserted);
+ if (inserted)
+ new (&it->getMapped()) Mapped();
+
+ return it->getMapped();
+ }
+};
+
+
+template <typename Key, typename Mapped, typename Allocator = HashTableAllocator>
+using FixedImplicitZeroHashMap = FixedHashMap<
+ Key,
+ Mapped,
+ FixedHashMapImplicitZeroCell<Key, Mapped>,
+ FixedHashTableStoredSize<FixedHashMapImplicitZeroCell<Key, Mapped>>,
+ Allocator>;
+
+template <typename Key, typename Mapped, typename Allocator = HashTableAllocator>
+using FixedImplicitZeroHashMapWithCalculatedSize = FixedHashMap<
+ Key,
+ Mapped,
+ FixedHashMapImplicitZeroCell<Key, Mapped>,
+ FixedHashTableCalculatedSize<FixedHashMapImplicitZeroCell<Key, Mapped>>,
+ Allocator>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/FixedHashTable.h b/ydb/library/arrow_clickhouse/Common/HashTable/FixedHashTable.h
new file mode 100644
index 00000000000..2e8d781fc7a
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/FixedHashTable.h
@@ -0,0 +1,497 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <Common/HashTable/HashTable.h>
+
+namespace CH
+{
+
+template <typename Key, typename TState = HashTableNoState>
+struct FixedHashTableCell
+{
+ using State = TState;
+
+ using value_type = Key;
+ using mapped_type = VoidMapped;
+ bool full;
+
+ FixedHashTableCell() {} //-V730
+ FixedHashTableCell(const Key &, const State &) : full(true) {}
+
+ const VoidKey getKey() const { return {}; }
+ VoidMapped getMapped() const { return {}; }
+
+ bool isZero(const State &) const { return !full; }
+ void setZero() { full = false; }
+ static constexpr bool need_zero_value_storage = false;
+
+ /// This Cell is only stored inside an iterator. It's used to accommodate the fact
+ /// that the iterator based API always provide a reference to a continuous memory
+ /// containing the Key. As a result, we have to instantiate a real Key field.
+ /// All methods that return a mutable reference to the Key field are named with
+ /// -Mutable suffix, indicating this is uncommon usage. As this is only for lookup
+ /// tables, it's totally fine to discard the Key mutations.
+ struct CellExt
+ {
+ Key key;
+
+ const VoidKey getKey() const { return {}; }
+ VoidMapped getMapped() const { return {}; }
+ const value_type & getValue() const { return key; }
+ void update(Key && key_, FixedHashTableCell *) { key = key_; }
+ };
+};
+
+
+/// How to obtain the size of the table.
+
+template <typename Cell>
+struct FixedHashTableStoredSize
+{
+ size_t m_size = 0;
+
+ size_t getSize(const Cell *, const typename Cell::State &, size_t) const { return m_size; }
+ bool isEmpty(const Cell *, const typename Cell::State &, size_t) const { return m_size == 0; }
+
+ void increaseSize() { ++m_size; }
+ void clearSize() { m_size = 0; }
+ void setSize(size_t to) { m_size = to; }
+};
+
+template <typename Cell>
+struct FixedHashTableCalculatedSize
+{
+ size_t getSize(const Cell * buf, const typename Cell::State & state, size_t num_cells) const
+ {
+ size_t res = 0;
+ for (const Cell * end = buf + num_cells; buf != end; ++buf)
+ if (!buf->isZero(state))
+ ++res;
+ return res;
+ }
+
+ bool isEmpty(const Cell * buf, const typename Cell::State & state, size_t num_cells) const
+ {
+ for (const Cell * end = buf + num_cells; buf != end; ++buf)
+ if (!buf->isZero(state))
+ return false;
+ return true;
+ }
+
+ void increaseSize() {}
+ void clearSize() {}
+ void setSize(size_t) {}
+};
+
+
+/** Used as a lookup table for small keys such as UInt8, UInt16. It's different
+ * than a HashTable in that keys are not stored in the Cell buf, but inferred
+ * inside each iterator. There are a bunch of to make it faster than using
+ * HashTable: a) It doesn't have a conflict chain; b) There is no key
+ * comparison; c) The number of cycles for checking cell empty is halved; d)
+ * Memory layout is tighter, especially the Clearable variants.
+ *
+ * NOTE: For Set variants this should always be better. For Map variants
+ * however, as we need to assemble the real cell inside each iterator, there
+ * might be some cases we fall short.
+ *
+ * TODO: Deprecate the cell API so that end users don't rely on the structure
+ * of cell. Instead iterator should be used for operations such as cell
+ * transfer, key updates (f.g. StringRef) and serde. This will allow
+ * TwoLevelHashSet(Map) to contain different type of sets(maps).
+ */
+template <typename Key, typename Cell, typename Size, typename Allocator>
+class FixedHashTable : /*private boost::noncopyable,*/ protected Allocator, protected Cell::State, protected Size
+{
+ static constexpr size_t NUM_CELLS = 1ULL << (sizeof(Key) * 8);
+
+protected:
+ friend class const_iterator;
+ friend class iterator;
+ friend class Reader;
+
+ using Self = FixedHashTable;
+
+ Cell * buf; /// A piece of memory for all elements.
+
+ void alloc() { buf = reinterpret_cast<Cell *>(Allocator::alloc(NUM_CELLS * sizeof(Cell))); }
+
+ void free()
+ {
+ if (buf)
+ {
+ Allocator::free(buf, getBufferSizeInBytes());
+ buf = nullptr;
+ }
+ }
+
+ void destroyElements()
+ {
+ if (!std::is_trivially_destructible_v<Cell>)
+ for (iterator it = begin(), it_end = end(); it != it_end; ++it)
+ it.ptr->~Cell();
+ }
+
+
+ template <typename Derived, bool is_const>
+ class iterator_base
+ {
+ using Container = std::conditional_t<is_const, const Self, Self>;
+ using cell_type = std::conditional_t<is_const, const Cell, Cell>;
+
+ Container * container;
+ cell_type * ptr;
+
+ friend class FixedHashTable;
+
+ public:
+ iterator_base() {}
+ iterator_base(Container * container_, cell_type * ptr_) : container(container_), ptr(ptr_)
+ {
+ cell.update(ptr - container->buf, ptr);
+ }
+
+ bool operator==(const iterator_base & rhs) const { return ptr == rhs.ptr; }
+ bool operator!=(const iterator_base & rhs) const { return ptr != rhs.ptr; }
+
+ Derived & operator++()
+ {
+ ++ptr;
+
+ /// Skip empty cells in the main buffer.
+ auto buf_end = container->buf + container->NUM_CELLS;
+ while (ptr < buf_end && ptr->isZero(*container))
+ ++ptr;
+
+ return static_cast<Derived &>(*this);
+ }
+
+ auto & operator*()
+ {
+ if (cell.key != ptr - container->buf)
+ cell.update(ptr - container->buf, ptr);
+ return cell;
+ }
+ auto * operator-> ()
+ {
+ if (cell.key != ptr - container->buf)
+ cell.update(ptr - container->buf, ptr);
+ return &cell;
+ }
+
+ auto getPtr() const { return ptr; }
+ size_t getHash() const { return ptr - container->buf; }
+ size_t getCollisionChainLength() const { return 0; }
+ typename cell_type::CellExt cell;
+ };
+
+
+public:
+ using key_type = Key;
+ using mapped_type = typename Cell::mapped_type;
+ using value_type = typename Cell::value_type;
+ using cell_type = Cell;
+
+ using LookupResult = Cell *;
+ using ConstLookupResult = const Cell *;
+
+
+ size_t hash(const Key & x) const { return x; }
+
+ FixedHashTable() { alloc(); }
+
+ FixedHashTable(FixedHashTable && rhs) : buf(nullptr) { *this = std::move(rhs); }
+
+ ~FixedHashTable()
+ {
+ destroyElements();
+ free();
+ }
+
+ FixedHashTable & operator=(FixedHashTable && rhs)
+ {
+ destroyElements();
+ free();
+
+ std::swap(buf, rhs.buf);
+ this->setSize(rhs.size());
+
+ Allocator::operator=(std::move(rhs));
+ Cell::State::operator=(std::move(rhs));
+
+ return *this;
+ }
+#if 0
+ class Reader final : private Cell::State
+ {
+ public:
+ Reader(DB::ReadBuffer & in_) : in(in_) {}
+
+ Reader(const Reader &) = delete;
+ Reader & operator=(const Reader &) = delete;
+
+ bool next()
+ {
+ if (!is_initialized)
+ {
+ Cell::State::read(in);
+ DB::readVarUInt(size, in);
+ is_initialized = true;
+ }
+
+ if (read_count == size)
+ {
+ is_eof = true;
+ return false;
+ }
+
+ cell.read(in);
+ ++read_count;
+
+ return true;
+ }
+
+ inline const value_type & get() const
+ {
+ if (!is_initialized || is_eof)
+ throw CH::Exception("No available data");
+
+ return cell.getValue();
+ }
+
+ private:
+ DB::ReadBuffer & in;
+ Cell cell;
+ size_t read_count = 0;
+ size_t size = 0;
+ bool is_eof = false;
+ bool is_initialized = false;
+ };
+#endif
+
+ class iterator : public iterator_base<iterator, false>
+ {
+ public:
+ using iterator_base<iterator, false>::iterator_base;
+ };
+
+ class const_iterator : public iterator_base<const_iterator, true>
+ {
+ public:
+ using iterator_base<const_iterator, true>::iterator_base;
+ };
+
+
+ const_iterator begin() const
+ {
+ if (!buf)
+ return end();
+
+ const Cell * ptr = buf;
+ auto buf_end = buf + NUM_CELLS;
+ while (ptr < buf_end && ptr->isZero(*this))
+ ++ptr;
+
+ return const_iterator(this, ptr);
+ }
+
+ const_iterator cbegin() const { return begin(); }
+
+ iterator begin()
+ {
+ if (!buf)
+ return end();
+
+ Cell * ptr = buf;
+ auto buf_end = buf + NUM_CELLS;
+ while (ptr < buf_end && ptr->isZero(*this))
+ ++ptr;
+
+ return iterator(this, ptr);
+ }
+
+ const_iterator end() const
+ {
+ /// Avoid UBSan warning about adding zero to nullptr. It is valid in C++20 (and earlier) but not valid in C.
+ return const_iterator(this, buf ? buf + NUM_CELLS : buf);
+ }
+
+ const_iterator cend() const
+ {
+ return end();
+ }
+
+ iterator end()
+ {
+ return iterator(this, buf ? buf + NUM_CELLS : buf);
+ }
+
+
+public:
+ /// The last parameter is unused but exists for compatibility with HashTable interface.
+ void ALWAYS_INLINE emplace(const Key & x, LookupResult & it, bool & inserted, size_t /* hash */ = 0)
+ {
+ it = &buf[x];
+
+ if (!buf[x].isZero(*this))
+ {
+ inserted = false;
+ return;
+ }
+
+ new (&buf[x]) Cell(x, *this);
+ inserted = true;
+ this->increaseSize();
+ }
+
+ std::pair<LookupResult, bool> ALWAYS_INLINE insert(const value_type & x)
+ {
+ std::pair<LookupResult, bool> res;
+ emplace(Cell::getKey(x), res.first, res.second);
+ if (res.second)
+ insertSetMapped(res.first->getMapped(), x);
+
+ return res;
+ }
+
+ LookupResult ALWAYS_INLINE find(const Key & x) { return !buf[x].isZero(*this) ? &buf[x] : nullptr; }
+
+ ConstLookupResult ALWAYS_INLINE find(const Key & x) const { return const_cast<std::decay_t<decltype(*this)> *>(this)->find(x); }
+
+ LookupResult ALWAYS_INLINE find(const Key &, size_t hash_value) { return !buf[hash_value].isZero(*this) ? &buf[hash_value] : nullptr; }
+
+ ConstLookupResult ALWAYS_INLINE find(const Key & key, size_t hash_value) const
+ {
+ return const_cast<std::decay_t<decltype(*this)> *>(this)->find(key, hash_value);
+ }
+
+ bool ALWAYS_INLINE has(const Key & x) const { return !buf[x].isZero(*this); }
+ bool ALWAYS_INLINE has(const Key &, size_t hash_value) const { return !buf[hash_value].isZero(*this); }
+#if 0
+ void write(DB::WriteBuffer & wb) const
+ {
+ Cell::State::write(wb);
+ DB::writeVarUInt(size(), wb);
+
+ if (!buf)
+ return;
+
+ for (auto ptr = buf, buf_end = buf + NUM_CELLS; ptr < buf_end; ++ptr)
+ {
+ if (!ptr->isZero(*this))
+ {
+ DB::writeVarUInt(ptr - buf);
+ ptr->write(wb);
+ }
+ }
+ }
+
+ void writeText(DB::WriteBuffer & wb) const
+ {
+ Cell::State::writeText(wb);
+ DB::writeText(size(), wb);
+
+ if (!buf)
+ return;
+
+ for (auto ptr = buf, buf_end = buf + NUM_CELLS; ptr < buf_end; ++ptr)
+ {
+ if (!ptr->isZero(*this))
+ {
+ DB::writeChar(',', wb);
+ DB::writeText(ptr - buf, wb);
+ DB::writeChar(',', wb);
+ ptr->writeText(wb);
+ }
+ }
+ }
+
+ void read(DB::ReadBuffer & rb)
+ {
+ Cell::State::read(rb);
+ destroyElements();
+ size_t m_size;
+ DB::readVarUInt(m_size, rb);
+ this->setSize(m_size);
+ free();
+ alloc();
+
+ for (size_t i = 0; i < m_size; ++i)
+ {
+ size_t place_value = 0;
+ DB::readVarUInt(place_value, rb);
+ Cell x;
+ x.read(rb);
+ new (&buf[place_value]) Cell(x, *this);
+ }
+ }
+
+ void readText(DB::ReadBuffer & rb)
+ {
+ Cell::State::readText(rb);
+ destroyElements();
+ size_t m_size;
+ DB::readText(m_size, rb);
+ this->setSize(m_size);
+ free();
+ alloc();
+
+ for (size_t i = 0; i < m_size; ++i)
+ {
+ size_t place_value = 0;
+ DB::assertChar(',', rb);
+ DB::readText(place_value, rb);
+ Cell x;
+ DB::assertChar(',', rb);
+ x.readText(rb);
+ new (&buf[place_value]) Cell(x, *this);
+ }
+ }
+#endif
+ size_t size() const { return this->getSize(buf, *this, NUM_CELLS); }
+ bool empty() const { return this->isEmpty(buf, *this, NUM_CELLS); }
+
+ void clear()
+ {
+ destroyElements();
+ this->clearSize();
+
+ memset(static_cast<void *>(buf), 0, NUM_CELLS * sizeof(*buf));
+ }
+
+ /// After executing this function, the table can only be destroyed,
+ /// and also you can use the methods `size`, `empty`, `begin`, `end`.
+ void clearAndShrink()
+ {
+ destroyElements();
+ this->clearSize();
+ free();
+ }
+
+ size_t getBufferSizeInBytes() const { return NUM_CELLS * sizeof(Cell); }
+
+ size_t getBufferSizeInCells() const { return NUM_CELLS; }
+
+ /// Return offset for result in internal buffer.
+ /// Result can have value up to `getBufferSizeInCells() + 1`
+ /// because offset for zero value considered to be 0
+ /// and for other values it will be `offset in buffer + 1`
+ size_t offsetInternal(ConstLookupResult ptr) const
+ {
+ if (ptr->isZero(*this))
+ return 0;
+ return ptr - buf + 1;
+ }
+
+ const Cell * data() const { return buf; }
+ Cell * data() { return buf; }
+
+#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
+ size_t getCollisions() const { return 0; }
+#endif
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/Hash.h b/ydb/library/arrow_clickhouse/Common/HashTable/Hash.h
new file mode 100644
index 00000000000..89aaeb48a48
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/Hash.h
@@ -0,0 +1,417 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <util/digest/city.h>
+#include <common/types.h>
+#include <common/unaligned.h>
+#include <common/StringRef.h>
+
+#include <type_traits>
+
+namespace CH
+{
+
+/** Hash functions that are better than the trivial function std::hash.
+ *
+ * Example: when we do aggregation by the visitor ID, the performance increase is more than 5 times.
+ * This is because of following reasons:
+ * - in Yandex, visitor identifier is an integer that has timestamp with seconds resolution in lower bits;
+ * - in typical implementation of standard library, hash function for integers is trivial and just use lower bits;
+ * - traffic is non-uniformly distributed across a day;
+ * - we are using open-addressing linear probing hash tables that are most critical to hash function quality,
+ * and trivial hash function gives disastrous results.
+ */
+
+/** Taken from MurmurHash. This is Murmur finalizer.
+ * Faster than intHash32 when inserting into the hash table UInt64 -> UInt64, where the key is the visitor ID.
+ */
+inline UInt64 intHash64(UInt64 x)
+{
+ x ^= x >> 33;
+ x *= 0xff51afd7ed558ccdULL;
+ x ^= x >> 33;
+ x *= 0xc4ceb9fe1a85ec53ULL;
+ x ^= x >> 33;
+
+ return x;
+}
+
+/** CRC32C is not very high-quality as a hash function,
+ * according to avalanche and bit independence tests (see SMHasher software), as well as a small number of bits,
+ * but can behave well when used in hash tables,
+ * due to high speed (latency 3 + 1 clock cycle, throughput 1 clock cycle).
+ * Works only with SSE 4.2 support.
+ */
+#ifdef __SSE4_2__
+#include <nmmintrin.h>
+#endif
+
+#if defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
+#include <arm_acle.h>
+#include <arm_neon.h>
+#endif
+
+inline UInt64 intHashCRC32(UInt64 x)
+{
+#ifdef __SSE4_2__
+ return _mm_crc32_u64(-1ULL, x);
+#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
+ return __crc32cd(-1U, x);
+#else
+ /// On other platforms we do not have CRC32. NOTE This can be confusing.
+ return intHash64(x);
+#endif
+}
+
+inline UInt64 intHashCRC32(UInt64 x, UInt64 updated_value)
+{
+#ifdef __SSE4_2__
+ return _mm_crc32_u64(updated_value, x);
+#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
+ return __crc32cd(updated_value, x);
+#else
+ /// On other platforms we do not have CRC32. NOTE This can be confusing.
+ return intHash64(x) ^ updated_value;
+#endif
+}
+
+template <typename T>
+inline typename std::enable_if<(sizeof(T) > sizeof(UInt64)), UInt64>::type
+intHashCRC32(const T & x, UInt64 updated_value)
+{
+ auto * begin = reinterpret_cast<const char *>(&x);
+ for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64))
+ {
+ updated_value = intHashCRC32(unalignedLoad<UInt64>(begin), updated_value);
+ begin += sizeof(UInt64);
+ }
+
+ return updated_value;
+}
+
+
+inline UInt32 updateWeakHash32(const UInt8 * pos, size_t size, UInt32 updated_value)
+{
+ if (size < 8)
+ {
+ UInt64 value = 0;
+
+ switch (size)
+ {
+ case 0:
+ break;
+ case 1:
+ __builtin_memcpy(&value, pos, 1);
+ break;
+ case 2:
+ __builtin_memcpy(&value, pos, 2);
+ break;
+ case 3:
+ __builtin_memcpy(&value, pos, 3);
+ break;
+ case 4:
+ __builtin_memcpy(&value, pos, 4);
+ break;
+ case 5:
+ __builtin_memcpy(&value, pos, 5);
+ break;
+ case 6:
+ __builtin_memcpy(&value, pos, 6);
+ break;
+ case 7:
+ __builtin_memcpy(&value, pos, 7);
+ break;
+ default:
+ __builtin_unreachable();
+ }
+
+ reinterpret_cast<unsigned char *>(&value)[7] = size;
+ return intHashCRC32(value, updated_value);
+ }
+
+ const auto * end = pos + size;
+ while (pos + 8 <= end)
+ {
+ auto word = unalignedLoad<UInt64>(pos);
+ updated_value = intHashCRC32(word, updated_value);
+
+ pos += 8;
+ }
+
+ if (pos < end)
+ {
+ /// If string size is not divisible by 8.
+ /// Lets' assume the string was 'abcdefghXYZ', so it's tail is 'XYZ'.
+ UInt8 tail_size = end - pos;
+ /// Load tailing 8 bytes. Word is 'defghXYZ'.
+ auto word = unalignedLoad<UInt64>(end - 8);
+ /// Prepare mask which will set other 5 bytes to 0. It is 0xFFFFFFFFFFFFFFFF << 5 = 0xFFFFFF0000000000.
+ /// word & mask = '\0\0\0\0\0XYZ' (bytes are reversed because of little ending)
+ word &= (~UInt64(0)) << UInt8(8 * (8 - tail_size));
+ /// Use least byte to store tail length.
+ word |= tail_size;
+ /// Now word is '\3\0\0\0\0XYZ'
+ updated_value = intHashCRC32(word, updated_value);
+ }
+
+ return updated_value;
+}
+
+template <typename T>
+inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) <= sizeof(UInt64)), T> key)
+{
+ union
+ {
+ T in;
+ UInt64 out;
+ } u;
+ u.out = 0;
+ u.in = key;
+ return intHash64(u.out);
+}
+
+
+template <typename T>
+inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> key)
+{
+ if constexpr (is_big_int_v<T> && sizeof(T) == 16)
+ {
+ /// TODO This is classical antipattern.
+ return intHash64(
+ static_cast<UInt64>(key) ^
+ static_cast<UInt64>(key >> 64));
+ }
+ else if constexpr (is_big_int_v<T> && sizeof(T) == 32)
+ {
+ return intHash64(
+ static_cast<UInt64>(key) ^
+ static_cast<UInt64>(key >> 64) ^
+ static_cast<UInt64>(key >> 128) ^
+ static_cast<UInt64>(key >> 256));
+ }
+ assert(false);
+ __builtin_unreachable();
+}
+
+
+template <typename T>
+struct DefaultHash
+{
+ size_t operator() (T key) const
+ {
+ return DefaultHash64<T>(key);
+ }
+};
+
+template <typename T> struct HashCRC32;
+
+template <typename T>
+inline size_t hashCRC32(std::enable_if_t<(sizeof(T) <= sizeof(UInt64)), T> key)
+{
+ union
+ {
+ T in;
+ UInt64 out;
+ } u;
+ u.out = 0;
+ u.in = key;
+ return intHashCRC32(u.out);
+}
+
+template <typename T>
+inline size_t hashCRC32(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> key)
+{
+ return intHashCRC32(key, -1);
+}
+
+#define DEFINE_HASH(T) \
+template <> struct HashCRC32<T>\
+{\
+ size_t operator() (T key) const\
+ {\
+ return hashCRC32<T>(key);\
+ }\
+};
+
+DEFINE_HASH(UInt8)
+DEFINE_HASH(UInt16)
+DEFINE_HASH(UInt32)
+DEFINE_HASH(UInt64)
+DEFINE_HASH(UInt128)
+DEFINE_HASH(UInt256)
+DEFINE_HASH(Int8)
+DEFINE_HASH(Int16)
+DEFINE_HASH(Int32)
+DEFINE_HASH(Int64)
+DEFINE_HASH(Int128)
+DEFINE_HASH(Int256)
+DEFINE_HASH(float)
+DEFINE_HASH(double)
+
+#undef DEFINE_HASH
+
+
+struct UInt128Hash
+{
+ size_t operator()(UInt128 x) const
+ {
+ return ::Hash128to64({x.items[0], x.items[1]});
+ }
+};
+#if 0
+struct UUIDHash
+{
+ size_t operator()(DB::UUID x) const
+ {
+ return UInt128Hash()(x.toUnderType());
+ }
+};
+#endif
+#ifdef __SSE4_2__
+
+struct UInt128HashCRC32
+{
+ size_t operator()(UInt128 x) const
+ {
+ UInt64 crc = -1ULL;
+ crc = _mm_crc32_u64(crc, x.items[0]);
+ crc = _mm_crc32_u64(crc, x.items[1]);
+ return crc;
+ }
+};
+
+#else
+
+/// On other platforms we do not use CRC32. NOTE This can be confusing.
+struct UInt128HashCRC32 : public UInt128Hash {};
+
+#endif
+
+struct UInt128TrivialHash
+{
+ size_t operator()(UInt128 x) const { return x.items[0]; }
+};
+#if 0
+struct UUIDTrivialHash
+{
+ size_t operator()(DB::UUID x) const { return x.toUnderType().items[0]; }
+};
+#endif
+struct UInt256Hash
+{
+ size_t operator()(UInt256 x) const
+ {
+ /// NOTE suboptimal
+ return ::Hash128to64({
+ ::Hash128to64({x.items[0], x.items[1]}),
+ ::Hash128to64({x.items[2], x.items[3]})});
+ }
+};
+
+#ifdef __SSE4_2__
+
+struct UInt256HashCRC32
+{
+ size_t operator()(UInt256 x) const
+ {
+ UInt64 crc = -1ULL;
+ crc = _mm_crc32_u64(crc, x.items[0]);
+ crc = _mm_crc32_u64(crc, x.items[1]);
+ crc = _mm_crc32_u64(crc, x.items[2]);
+ crc = _mm_crc32_u64(crc, x.items[3]);
+ return crc;
+ }
+};
+
+#else
+
+/// We do not need to use CRC32 on other platforms. NOTE This can be confusing.
+struct UInt256HashCRC32 : public UInt256Hash {};
+
+#endif
+
+template <>
+struct DefaultHash<UInt128> : public UInt128Hash {};
+
+template <>
+struct DefaultHash<UInt256> : public UInt256Hash {};
+#if 0
+template <>
+struct DefaultHash<DB::UUID> : public UUIDHash {};
+#endif
+
+/// It is reasonable to use for UInt8, UInt16 with sufficient hash table size.
+struct TrivialHash
+{
+ template <typename T>
+ size_t operator() (T key) const
+ {
+ return key;
+ }
+};
+
+
+/** A relatively good non-cryptographic hash function from UInt64 to UInt32.
+ * But worse (both in quality and speed) than just cutting intHash64.
+ * Taken from here: http://www.concentric.net/~ttwang/tech/inthash.htm
+ *
+ * Slightly changed compared to the function by link: shifts to the right are accidentally replaced by a cyclic shift to the right.
+ * This change did not affect the smhasher test results.
+ *
+ * It is recommended to use different salt for different tasks.
+ * That was the case that in the database values were sorted by hash (for low-quality pseudo-random spread),
+ * and in another place, in the aggregate function, the same hash was used in the hash table,
+ * as a result, this aggregate function was monstrously slowed due to collisions.
+ *
+ * NOTE Salting is far from perfect, because it commutes with first steps of calculation.
+ *
+ * NOTE As mentioned, this function is slower than intHash64.
+ * But occasionally, it is faster, when written in a loop and loop is vectorized.
+ */
+template <UInt64 salt>
+inline UInt32 intHash32(UInt64 key)
+{
+ key ^= salt;
+
+ key = (~key) + (key << 18);
+ key = key ^ ((key >> 31) | (key << 33));
+ key = key * 21;
+ key = key ^ ((key >> 11) | (key << 53));
+ key = key + (key << 6);
+ key = key ^ ((key >> 22) | (key << 42));
+
+ return key;
+}
+
+
+/// For containers.
+template <typename T, UInt64 salt = 0>
+struct IntHash32
+{
+ size_t operator() (const T & key) const
+ {
+ if constexpr (is_big_int_v<T> && sizeof(T) == 16)
+ {
+ return intHash32<salt>(key.items[0] ^ key.items[1]);
+ }
+ else if constexpr (is_big_int_v<T> && sizeof(T) == 32)
+ {
+ return intHash32<salt>(key.items[0] ^ key.items[1] ^ key.items[2] ^ key.items[3]);
+ }
+ else if constexpr (sizeof(T) <= sizeof(UInt64))
+ {
+ return intHash32<salt>(key);
+ }
+
+ assert(false);
+ __builtin_unreachable();
+ }
+};
+
+template <>
+struct DefaultHash<CH::StringRef> : public CH::StringRefHash {};
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/HashMap.h b/ydb/library/arrow_clickhouse/Common/HashTable/HashMap.h
new file mode 100644
index 00000000000..a76cd5b353d
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/HashMap.h
@@ -0,0 +1,314 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <Common/HashTable/Hash.h>
+#include <Common/HashTable/HashTable.h>
+#include <Common/HashTable/HashTableAllocator.h>
+
+
+/** NOTE HashMap could only be used for memmoveable (position independent) types.
+ * Example: std::string is not position independent in libstdc++ with C++11 ABI or in libc++.
+ * Also, key in hash table must be of type, that zero bytes is compared equals to zero key.
+ */
+
+namespace CH
+{
+
+struct NoInitTag
+{
+};
+
+/// A pair that does not initialize the elements, if not needed.
+template <typename First, typename Second>
+struct PairNoInit
+{
+ First first;
+ Second second;
+
+ PairNoInit() {} /// NOLINT
+
+ template <typename FirstValue>
+ PairNoInit(FirstValue && first_, NoInitTag)
+ : first(std::forward<FirstValue>(first_))
+ {
+ }
+
+ template <typename FirstValue, typename SecondValue>
+ PairNoInit(FirstValue && first_, SecondValue && second_)
+ : first(std::forward<FirstValue>(first_))
+ , second(std::forward<SecondValue>(second_))
+ {
+ }
+};
+
+template <typename First, typename Second>
+PairNoInit<std::decay_t<First>, std::decay_t<Second>> makePairNoInit(First && first, Second && second)
+{
+ return PairNoInit<std::decay_t<First>, std::decay_t<Second>>(std::forward<First>(first), std::forward<Second>(second));
+}
+
+
+template <typename Key, typename TMapped, typename Hash, typename TState = HashTableNoState>
+struct HashMapCell
+{
+ using Mapped = TMapped;
+ using State = TState;
+
+ using value_type = PairNoInit<Key, Mapped>;
+ using mapped_type = Mapped;
+ using key_type = Key;
+
+ value_type value;
+
+ HashMapCell() = default;
+ HashMapCell(const Key & key_, const State &) : value(key_, NoInitTag()) {}
+ HashMapCell(const value_type & value_, const State &) : value(value_) {}
+
+ /// Get the key (externally).
+ const Key & getKey() const { return value.first; }
+ Mapped & getMapped() { return value.second; }
+ const Mapped & getMapped() const { return value.second; }
+ const value_type & getValue() const { return value; }
+
+ /// Get the key (internally).
+ static const Key & getKey(const value_type & value) { return value.first; }
+
+ bool keyEquals(const Key & key_) const { return bitEquals(value.first, key_); }
+ bool keyEquals(const Key & key_, size_t /*hash_*/) const { return bitEquals(value.first, key_); }
+ bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return bitEquals(value.first, key_); }
+
+ void setHash(size_t /*hash_value*/) {}
+ size_t getHash(const Hash & hash) const { return hash(value.first); }
+
+ bool isZero(const State & state) const { return isZero(value.first, state); }
+ static bool isZero(const Key & key, const State & /*state*/) { return ZeroTraits::check(key); }
+
+ /// Set the key value to zero.
+ void setZero() { ZeroTraits::set(value.first); }
+
+ /// Do I need to store the zero key separately (that is, can a zero key be inserted into the hash table).
+ static constexpr bool need_zero_value_storage = true;
+
+ void setMapped(const value_type & value_) { value.second = value_.second; }
+#if 0
+ /// Serialization, in binary and text form.
+ void write(DB::WriteBuffer & wb) const
+ {
+ DB::writeBinary(value.first, wb);
+ DB::writeBinary(value.second, wb);
+ }
+
+ void writeText(DB::WriteBuffer & wb) const
+ {
+ DB::writeDoubleQuoted(value.first, wb);
+ DB::writeChar(',', wb);
+ DB::writeDoubleQuoted(value.second, wb);
+ }
+
+ /// Deserialization, in binary and text form.
+ void read(DB::ReadBuffer & rb)
+ {
+ DB::readBinary(value.first, rb);
+ DB::readBinary(value.second, rb);
+ }
+
+ void readText(DB::ReadBuffer & rb)
+ {
+ DB::readDoubleQuoted(value.first, rb);
+ DB::assertChar(',', rb);
+ DB::readDoubleQuoted(value.second, rb);
+ }
+#endif
+ static bool constexpr need_to_notify_cell_during_move = false;
+
+ static void move(HashMapCell * /* old_location */, HashMapCell * /* new_location */) {}
+
+ template <size_t I>
+ auto & get() & {
+ if constexpr (I == 0) return value.first;
+ else if constexpr (I == 1) return value.second;
+ }
+
+ template <size_t I>
+ auto const & get() const & {
+ if constexpr (I == 0) return value.first;
+ else if constexpr (I == 1) return value.second;
+ }
+
+ template <size_t I>
+ auto && get() && {
+ if constexpr (I == 0) return std::move(value.first);
+ else if constexpr (I == 1) return std::move(value.second);
+ }
+
+};
+
+template <typename Key, typename TMapped, typename Hash, typename TState = HashTableNoState>
+struct HashMapCellWithSavedHash : public HashMapCell<Key, TMapped, Hash, TState>
+{
+ using Base = HashMapCell<Key, TMapped, Hash, TState>;
+
+ size_t saved_hash;
+
+ using Base::Base;
+
+ bool keyEquals(const Key & key_) const { return bitEquals(this->value.first, key_); }
+ bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && bitEquals(this->value.first, key_); }
+ bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); }
+
+ void setHash(size_t hash_value) { saved_hash = hash_value; }
+ size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
+};
+
+template <
+ typename Key,
+ typename Cell,
+ typename Hash = DefaultHash<Key>,
+ typename Grower = HashTableGrower<>,
+ typename Allocator = HashTableAllocator>
+class HashMapTable : public HashTable<Key, Cell, Hash, Grower, Allocator>
+{
+public:
+ using Self = HashMapTable;
+ using Base = HashTable<Key, Cell, Hash, Grower, Allocator>;
+ using LookupResult = typename Base::LookupResult;
+
+ using Base::Base;
+
+ /// Merge every cell's value of current map into the destination map via emplace.
+ /// Func should have signature void(Mapped & dst, Mapped & src, bool emplaced).
+ /// Each filled cell in current map will invoke func once. If that map doesn't
+ /// have a key equals to the given cell, a new cell gets emplaced into that map,
+ /// and func is invoked with the third argument emplaced set to true. Otherwise
+ /// emplaced is set to false.
+ template <typename Func>
+ void ALWAYS_INLINE mergeToViaEmplace(Self & that, Func && func)
+ {
+ for (auto it = this->begin(), end = this->end(); it != end; ++it)
+ {
+ typename Self::LookupResult res_it;
+ bool inserted;
+ that.emplace(Cell::getKey(it->getValue()), res_it, inserted, it.getHash());
+ func(res_it->getMapped(), it->getMapped(), inserted);
+ }
+ }
+
+ /// Merge every cell's value of current map into the destination map via find.
+ /// Func should have signature void(Mapped & dst, Mapped & src, bool exist).
+ /// Each filled cell in current map will invoke func once. If that map doesn't
+ /// have a key equals to the given cell, func is invoked with the third argument
+ /// exist set to false. Otherwise exist is set to true.
+ template <typename Func>
+ void ALWAYS_INLINE mergeToViaFind(Self & that, Func && func)
+ {
+ for (auto it = this->begin(), end = this->end(); it != end; ++it)
+ {
+ auto res_it = that.find(Cell::getKey(it->getValue()), it.getHash());
+ if (!res_it)
+ func(it->getMapped(), it->getMapped(), false);
+ else
+ func(res_it->getMapped(), it->getMapped(), true);
+ }
+ }
+
+ /// Call func(const Key &, Mapped &) for each hash map element.
+ template <typename Func>
+ void forEachValue(Func && func)
+ {
+ for (auto & v : *this)
+ func(v.getKey(), v.getMapped());
+ }
+
+ /// Call func(Mapped &) for each hash map element.
+ template <typename Func>
+ void forEachMapped(Func && func)
+ {
+ for (auto & v : *this)
+ func(v.getMapped());
+ }
+
+ typename Cell::Mapped & ALWAYS_INLINE operator[](const Key & x)
+ {
+ LookupResult it;
+ bool inserted;
+ this->emplace(x, it, inserted);
+
+ /** It may seem that initialization is not necessary for POD-types (or __has_trivial_constructor),
+ * since the hash table memory is initially initialized with zeros.
+ * But, in fact, an empty cell may not be initialized with zeros in the following cases:
+ * - ZeroValueStorage (it only zeros the key);
+ * - after resizing and moving a part of the cells to the new half of the hash table, the old cells also have only the key to zero.
+ *
+ * On performance, there is almost always no difference, due to the fact that it->second is usually assigned immediately
+ * after calling `operator[]`, and since `operator[]` is inlined, the compiler removes unnecessary initialization.
+ *
+ * Sometimes due to initialization, the performance even grows. This occurs in code like `++map[key]`.
+ * When we do the initialization, for new cells, it's enough to make `store 1` right away.
+ * And if we did not initialize, then even though there was zero in the cell,
+ * the compiler can not guess about this, and generates the `load`, `increment`, `store` code.
+ */
+ if (inserted)
+ new (&it->getMapped()) typename Cell::Mapped();
+
+ return it->getMapped();
+ }
+};
+
+template <
+ typename Key,
+ typename Mapped,
+ typename Hash = DefaultHash<Key>,
+ typename Grower = HashTableGrower<>,
+ typename Allocator = HashTableAllocator>
+using HashMap = HashMapTable<Key, HashMapCell<Key, Mapped, Hash>, Hash, Grower, Allocator>;
+
+
+template <
+ typename Key,
+ typename Mapped,
+ typename Hash = DefaultHash<Key>,
+ typename Grower = HashTableGrower<>,
+ typename Allocator = HashTableAllocator>
+using HashMapWithSavedHash = HashMapTable<Key, HashMapCellWithSavedHash<Key, Mapped, Hash>, Hash, Grower, Allocator>;
+
+template <typename Key, typename Mapped, typename Hash,
+ size_t initial_size_degree>
+using HashMapWithStackMemory = HashMapTable<
+ Key,
+ HashMapCellWithSavedHash<Key, Mapped, Hash>,
+ Hash,
+ HashTableGrower<initial_size_degree>,
+ HashTableAllocatorWithStackMemory<
+ (1ULL << initial_size_degree)
+ * sizeof(HashMapCellWithSavedHash<Key, Mapped, Hash>)>>;
+
+}
+
+namespace std
+{
+
+ template <typename Key, typename TMapped, typename Hash, typename TState>
+ struct tuple_size<CH::HashMapCell<Key, TMapped, Hash, TState>> : std::integral_constant<size_t, 2> { };
+
+ template <typename Key, typename TMapped, typename Hash, typename TState>
+ struct tuple_element<0, CH::HashMapCell<Key, TMapped, Hash, TState>> { using type = Key; };
+
+ template <typename Key, typename TMapped, typename Hash, typename TState>
+ struct tuple_element<1, CH::HashMapCell<Key, TMapped, Hash, TState>> { using type = TMapped; };
+}
+
+namespace std
+{
+
+ template <typename Key, typename TMapped, typename Hash, typename TState>
+ struct tuple_size<CH::HashMapCellWithSavedHash<Key, TMapped, Hash, TState>> : std::integral_constant<size_t, 2> { };
+
+ template <typename Key, typename TMapped, typename Hash, typename TState>
+ struct tuple_element<0, CH::HashMapCellWithSavedHash<Key, TMapped, Hash, TState>> { using type = Key; };
+
+ template <typename Key, typename TMapped, typename Hash, typename TState>
+ struct tuple_element<1, CH::HashMapCellWithSavedHash<Key, TMapped, Hash, TState>> { using type = TMapped; };
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/HashSet.h b/ydb/library/arrow_clickhouse/Common/HashTable/HashSet.h
new file mode 100644
index 00000000000..0f349e5d9c3
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/HashSet.h
@@ -0,0 +1,124 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <Common/HashTable/Hash.h>
+#include <Common/HashTable/HashTable.h>
+#include <Common/HashTable/HashTableAllocator.h>
+
+/** NOTE HashSet could only be used for memmoveable (position independent) types.
+ * Example: std::string is not position independent in libstdc++ with C++11 ABI or in libc++.
+ * Also, key must be of type, that zero bytes is compared equals to zero key.
+ */
+
+namespace CH
+{
+
+template
+<
+ typename Key,
+ typename TCell,
+ typename Hash = DefaultHash<Key>,
+ typename Grower = HashTableGrower<>,
+ typename Allocator = HashTableAllocator
+>
+class HashSetTable : public HashTable<Key, TCell, Hash, Grower, Allocator>
+{
+public:
+ using Self = HashSetTable;
+ using Cell = TCell;
+
+ using Base = HashTable<Key, TCell, Hash, Grower, Allocator>;
+ using typename Base::LookupResult;
+
+ void merge(const Self & rhs)
+ {
+ if (!this->hasZero() && rhs.hasZero())
+ {
+ this->setHasZero();
+ ++this->m_size;
+ }
+
+ for (size_t i = 0; i < rhs.grower.bufSize(); ++i)
+ if (!rhs.buf[i].isZero(*this))
+ this->insert(rhs.buf[i].getValue());
+ }
+
+#if 0
+ void readAndMerge(DB::ReadBuffer & rb)
+ {
+ Cell::State::read(rb);
+
+ size_t new_size = 0;
+ DB::readVarUInt(new_size, rb);
+
+ this->resize(new_size);
+
+ for (size_t i = 0; i < new_size; ++i)
+ {
+ Cell x;
+ x.read(rb);
+ this->insert(x.getValue());
+ }
+ }
+#endif
+};
+
+
+template <typename Key, typename Hash, typename TState = HashTableNoState>
+struct HashSetCellWithSavedHash : public HashTableCell<Key, Hash, TState>
+{
+ using Base = HashTableCell<Key, Hash, TState>;
+
+ size_t saved_hash;
+
+ HashSetCellWithSavedHash() : Base() {} //-V730
+ HashSetCellWithSavedHash(const Key & key_, const typename Base::State & state) : Base(key_, state) {} //-V730
+
+ bool keyEquals(const Key & key_) const { return bitEquals(this->key, key_); }
+ bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && bitEquals(this->key, key_); }
+ bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); }
+
+ void setHash(size_t hash_value) { saved_hash = hash_value; }
+ size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
+};
+
+template
+<
+ typename Key,
+ typename Hash = DefaultHash<Key>,
+ typename Grower = HashTableGrower<>,
+ typename Allocator = HashTableAllocator
+>
+using HashSet = HashSetTable<Key, HashTableCell<Key, Hash>, Hash, Grower, Allocator>;
+
+template <typename Key, typename Hash, size_t initial_size_degree>
+using HashSetWithStackMemory = HashSet<
+ Key,
+ Hash,
+ HashTableGrower<initial_size_degree>,
+ HashTableAllocatorWithStackMemory<
+ (1ULL << initial_size_degree)
+ * sizeof(HashTableCell<Key, Hash>)>>;
+
+template
+<
+ typename Key,
+ typename Hash = DefaultHash<Key>,
+ typename Grower = HashTableGrower<>,
+ typename Allocator = HashTableAllocator
+>
+using HashSetWithSavedHash = HashSetTable<Key, HashSetCellWithSavedHash<Key, Hash>, Hash, Grower, Allocator>;
+
+template <typename Key, typename Hash, size_t initial_size_degree>
+using HashSetWithSavedHashWithStackMemory = HashSetWithSavedHash<
+ Key,
+ Hash,
+ HashTableGrower<initial_size_degree>,
+ HashTableAllocatorWithStackMemory<
+ (1ULL << initial_size_degree)
+ * sizeof(HashSetCellWithSavedHash<Key, Hash>)>>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/HashTable.h b/ydb/library/arrow_clickhouse/Common/HashTable/HashTable.h
new file mode 100644
index 00000000000..98fe66a0df3
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/HashTable.h
@@ -0,0 +1,1311 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <string.h>
+#include <math.h>
+
+#include <new>
+#include <utility>
+
+//#include <boost/noncopyable.hpp>
+
+#include <common/types.h>
+
+#include <Common/HashTable/HashTableAllocator.h>
+#include <Common/HashTable/HashTableKeyHolder.h>
+
+#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
+ #include <iostream>
+ #include <iomanip>
+#endif
+
+/** NOTE HashTable could only be used for memmoveable (position independent) types.
+ * Example: std::string is not position independent in libstdc++ with C++11 ABI or in libc++.
+ * Also, key in hash table must be of type, that zero bytes is compared equals to zero key.
+ */
+
+namespace CH
+{
+
+/** The state of the hash table that affects the properties of its cells.
+ * Used as a template parameter.
+ * For example, there is an implementation of an instantly clearable hash table - ClearableHashMap.
+ * For it, each cell holds the version number, and in the hash table itself is the current version.
+ * When clearing, the current version simply increases; All cells with a mismatching version are considered empty.
+ * Another example: for an approximate calculation of the number of unique visitors, there is a hash table for UniquesHashSet.
+ * It has the concept of "degree". At each overflow, cells with keys that do not divide by the corresponding power of the two are deleted.
+ */
+struct HashTableNoState
+{
+#if 0
+ /// Serialization, in binary and text form.
+ void write(DB::WriteBuffer &) const {}
+ void writeText(DB::WriteBuffer &) const {}
+
+ /// Deserialization, in binary and text form.
+ void read(DB::ReadBuffer &) {}
+ void readText(DB::ReadBuffer &) {}
+#endif
+};
+
+
+/// These functions can be overloaded for custom types.
+namespace ZeroTraits
+{
+
+template <typename T>
+bool check(const T x) { return x == T{}; }
+
+template <typename T>
+void set(T & x) { x = {}; }
+
+}
+
+
+/** Numbers are compared bitwise.
+ * Complex types are compared by operator== as usual (this is important if there are gaps).
+ *
+ * This is needed if you use floats as keys. They are compared by bit equality.
+ * Otherwise the invariants in hash table probing do not met when NaNs are present.
+ */
+template <typename T>
+inline bool bitEquals(T && a, T && b)
+{
+ using RealT = std::decay_t<T>;
+
+ if constexpr (std::is_floating_point_v<RealT>)
+ return 0 == memcmp(&a, &b, sizeof(RealT)); /// Note that memcmp with constant size is compiler builtin.
+ else
+ return a == b;
+}
+
+
+/**
+ * getKey/Mapped -- methods to get key/"mapped" values from the LookupResult returned by find() and
+ * emplace() methods of HashTable. Must not be called for a null LookupResult.
+ *
+ * We don't use iterators for lookup result. Instead, LookupResult is a pointer of some kind. There
+ * are methods getKey/Mapped, that return references or values to key/"mapped" values.
+ *
+ * Different hash table implementations support this interface to a varying degree:
+ *
+ * 1) Hash tables that store neither the key in its original form, nor a "mapped" value:
+ * FixedHashTable or StringHashTable. Neither GetKey nor GetMapped are supported, the only valid
+ * operation is checking LookupResult for null.
+ *
+ * 2) Hash maps that do not store the key, e.g. FixedHashMap or StringHashMap. Only GetMapped is
+ * supported.
+ *
+ * 3) Hash tables that store the key and do not have a "mapped" value, e.g. the normal HashTable.
+ * GetKey returns the key, and GetMapped returns a zero void pointer. This simplifies generic
+ * code that works with mapped values: it can overload on the return type of GetMapped(), and
+ * doesn't need other parameters. One example is insertSetMapped() function.
+ *
+ * 4) Hash tables that store both the key and the "mapped" value, e.g. HashMap. Both GetKey and
+ * GetMapped are supported.
+ *
+ * The implementation side goes as follows:
+ *
+ * for (1), LookupResult->getKey = const VoidKey, LookupResult->getMapped = VoidMapped;
+ *
+ * for (2), LookupResult->getKey = const VoidKey, LookupResult->getMapped = Mapped &;
+ *
+ * for (3) and (4), LookupResult->getKey = const Key [&], LookupResult->getMapped = Mapped &;
+ * VoidKey and VoidMapped may have specialized function overloads for generic code.
+ */
+
+struct VoidKey {};
+struct VoidMapped
+{
+ template <typename T>
+ auto & operator=(const T &)
+ {
+ return *this;
+ }
+};
+
+/** Compile-time interface for cell of the hash table.
+ * Different cell types are used to implement different hash tables.
+ * The cell must contain a key.
+ * It can also contain a value and arbitrary additional data
+ * (example: the stored hash value; version number for ClearableHashMap).
+ */
+template <typename Key, typename Hash, typename TState = HashTableNoState>
+struct HashTableCell
+{
+ using State = TState;
+
+ using key_type = Key;
+ using value_type = Key;
+ using mapped_type = VoidMapped;
+
+ Key key;
+
+ HashTableCell() {}
+
+ /// Create a cell with the given key / key and value.
+ HashTableCell(const Key & key_, const State &) : key(key_) {}
+
+ /// Get the key (externally).
+ const Key & getKey() const { return key; }
+ VoidMapped getMapped() const { return {}; }
+ const value_type & getValue() const { return key; }
+
+ /// Get the key (internally).
+ static const Key & getKey(const value_type & value) { return value; }
+
+ /// Are the keys at the cells equal?
+ bool keyEquals(const Key & key_) const { return bitEquals(key, key_); }
+ bool keyEquals(const Key & key_, size_t /*hash_*/) const { return bitEquals(key, key_); }
+ bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return bitEquals(key, key_); }
+
+ /// If the cell can remember the value of the hash function, then remember it.
+ void setHash(size_t /*hash_value*/) {}
+
+ /// If the cell can store the hash value in itself, then return the stored value.
+ /// It must be at least once calculated before.
+ /// If storing the hash value is not provided, then just compute the hash.
+ size_t getHash(const Hash & hash) const { return hash(key); }
+
+ /// Whether the key is zero. In the main buffer, cells with a zero key are considered empty.
+ /// If zero keys can be inserted into the table, then the cell for the zero key is stored separately, not in the main buffer.
+ /// Zero keys must be such that the zeroed-down piece of memory is a zero key.
+ bool isZero(const State & state) const { return isZero(key, state); }
+ static bool isZero(const Key & key, const State & /*state*/) { return ZeroTraits::check(key); }
+
+ /// Set the key value to zero.
+ void setZero() { ZeroTraits::set(key); }
+
+ /// Do the hash table need to store the zero key separately (that is, can a zero key be inserted into the hash table).
+ static constexpr bool need_zero_value_storage = true;
+
+ /// Set the mapped value, if any (for HashMap), to the corresponding `value`.
+ void setMapped(const value_type & /*value*/) {}
+#if 0
+ /// Serialization, in binary and text form.
+ void write(DB::WriteBuffer & wb) const { DB::writeBinary(key, wb); }
+ void writeText(DB::WriteBuffer & wb) const { DB::writeDoubleQuoted(key, wb); }
+
+ /// Deserialization, in binary and text form.
+ void read(DB::ReadBuffer & rb) { DB::readBinary(key, rb); }
+ void readText(DB::ReadBuffer & rb) { DB::readDoubleQuoted(key, rb); }
+#endif
+ /// When cell pointer is moved during erase, reinsert or resize operations
+
+ static constexpr bool need_to_notify_cell_during_move = false;
+
+ static void move(HashTableCell * /* old_location */, HashTableCell * /* new_location */) {}
+
+};
+
+/**
+ * A helper function for HashTable::insert() to set the "mapped" value.
+ * Overloaded on the mapped type, does nothing if it's VoidMapped.
+ */
+template <typename ValueType>
+void insertSetMapped(VoidMapped /* dest */, const ValueType & /* src */) {}
+
+template <typename MappedType, typename ValueType>
+void insertSetMapped(MappedType & dest, const ValueType & src) { dest = src.second; }
+
+
+/** Determines the size of the hash table, and when and how much it should be resized.
+ */
+template <size_t initial_size_degree = 8>
+struct HashTableGrower
+{
+ /// The state of this structure is enough to get the buffer size of the hash table.
+
+ UInt8 size_degree = initial_size_degree;
+ static constexpr auto initial_count = 1ULL << initial_size_degree;
+
+ /// If collision resolution chains are contiguous, we can implement erase operation by moving the elements.
+ static constexpr auto performs_linear_probing_with_single_step = true;
+
+ /// The size of the hash table in the cells.
+ size_t bufSize() const { return 1ULL << size_degree; }
+
+ size_t maxFill() const { return 1ULL << (size_degree - 1); }
+ size_t mask() const { return bufSize() - 1; }
+
+ /// From the hash value, get the cell number in the hash table.
+ size_t place(size_t x) const { return x & mask(); }
+
+ /// The next cell in the collision resolution chain.
+ size_t next(size_t pos) const { ++pos; return pos & mask(); }
+
+ /// Whether the hash table is sufficiently full. You need to increase the size of the hash table, or remove something unnecessary from it.
+ bool overflow(size_t elems) const { return elems > maxFill(); }
+
+ /// Increase the size of the hash table.
+ void increaseSize()
+ {
+ size_degree += size_degree >= 23 ? 1 : 2;
+ }
+
+ /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table.
+ void set(size_t num_elems)
+ {
+ size_degree = num_elems <= 1
+ ? initial_size_degree
+ : ((initial_size_degree > static_cast<size_t>(log2(num_elems - 1)) + 2)
+ ? initial_size_degree
+ : (static_cast<size_t>(log2(num_elems - 1)) + 2));
+ }
+
+ void setBufSize(size_t buf_size_)
+ {
+ size_degree = static_cast<size_t>(log2(buf_size_ - 1) + 1);
+ }
+};
+
+
+/** When used as a Grower, it turns a hash table into something like a lookup table.
+ * It remains non-optimal - the cells store the keys.
+ * Also, the compiler can not completely remove the code of passing through the collision resolution chain, although it is not needed.
+ * NOTE: Better to use FixedHashTable instead.
+ */
+template <size_t key_bits>
+struct HashTableFixedGrower
+{
+ static constexpr auto initial_count = 1ULL << key_bits;
+
+ static constexpr auto performs_linear_probing_with_single_step = true;
+
+ size_t bufSize() const { return 1ULL << key_bits; }
+ size_t place(size_t x) const { return x; }
+ /// You could write __builtin_unreachable(), but the compiler does not optimize everything, and it turns out less efficiently.
+ size_t next(size_t pos) const { return pos + 1; }
+ bool overflow(size_t /*elems*/) const { return false; }
+
+ void increaseSize() { __builtin_unreachable(); }
+ void set(size_t /*num_elems*/) {}
+ void setBufSize(size_t /*buf_size_*/) {}
+};
+
+
+/** If you want to store the zero key separately - a place to store it. */
+template <bool need_zero_value_storage, typename Cell>
+struct ZeroValueStorage;
+
+template <typename Cell>
+struct ZeroValueStorage<true, Cell> //-V730
+{
+private:
+ bool has_zero = false;
+ std::aligned_storage_t<sizeof(Cell), alignof(Cell)> zero_value_storage; /// Storage of element with zero key.
+
+public:
+ bool hasZero() const { return has_zero; }
+
+ void setHasZero()
+ {
+ has_zero = true;
+ new (zeroValue()) Cell();
+ }
+
+ void clearHasZero()
+ {
+ has_zero = false;
+ zeroValue()->~Cell();
+ }
+
+ Cell * zeroValue() { return std::launder(reinterpret_cast<Cell*>(&zero_value_storage)); }
+ const Cell * zeroValue() const { return std::launder(reinterpret_cast<const Cell*>(&zero_value_storage)); }
+};
+
+template <typename Cell>
+struct ZeroValueStorage<false, Cell>
+{
+ bool hasZero() const { return false; }
+ void setHasZero() { throw std::runtime_error("HashTable: logical error"); }
+ void clearHasZero() {}
+
+ Cell * zeroValue() { return nullptr; }
+ const Cell * zeroValue() const { return nullptr; }
+};
+
+
+template <bool enable, typename Allocator, typename Cell>
+struct AllocatorBufferDeleter;
+
+template <typename Allocator, typename Cell>
+struct AllocatorBufferDeleter<false, Allocator, Cell>
+{
+ AllocatorBufferDeleter(Allocator &, size_t) {}
+
+ void operator()(Cell *) const {}
+
+};
+
+template <typename Allocator, typename Cell>
+struct AllocatorBufferDeleter<true, Allocator, Cell>
+{
+ AllocatorBufferDeleter(Allocator & allocator_, size_t size_)
+ : allocator(allocator_)
+ , size(size_) {}
+
+ void operator()(Cell * buffer) const { allocator.free(buffer, size); }
+
+ Allocator & allocator;
+ size_t size;
+};
+
+
+// The HashTable
+template
+<
+ typename Key,
+ typename Cell,
+ typename Hash,
+ typename Grower,
+ typename Allocator
+>
+class HashTable :
+ //private boost::noncopyable,
+ protected Hash,
+ protected Allocator,
+ protected Cell::State,
+ protected ZeroValueStorage<Cell::need_zero_value_storage, Cell> /// empty base optimization
+{
+public:
+ // If we use an allocator with inline memory, check that the initial
+ // size of the hash table is in sync with the amount of this memory.
+ static constexpr size_t initial_buffer_bytes
+ = Grower::initial_count * sizeof(Cell);
+ static_assert(allocatorInitialBytes<Allocator> == 0
+ || allocatorInitialBytes<Allocator> == initial_buffer_bytes);
+
+protected:
+ friend class const_iterator;
+ friend class iterator;
+ friend class Reader;
+
+ template <typename, typename, typename, typename, typename, typename, size_t>
+ friend class TwoLevelHashTable;
+
+ template <typename, typename, size_t>
+ friend class TwoLevelStringHashTable;
+
+ template <typename SubMaps>
+ friend class StringHashTable;
+
+ using HashValue = size_t;
+ using Self = HashTable;
+
+ size_t m_size = 0; /// Amount of elements
+ Cell * buf; /// A piece of memory for all elements except the element with zero key.
+ Grower grower;
+
+#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
+ mutable size_t collisions = 0;
+#endif
+
+ /// Find a cell with the same key or an empty cell, starting from the specified position and further along the collision resolution chain.
+ size_t ALWAYS_INLINE findCell(const Key & x, size_t hash_value, size_t place_value) const
+ {
+ while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value, *this))
+ {
+ place_value = grower.next(place_value);
+#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
+ ++collisions;
+#endif
+ }
+
+ return place_value;
+ }
+
+
+ /// Find an empty cell, starting with the specified position and further along the collision resolution chain.
+ size_t ALWAYS_INLINE findEmptyCell(size_t place_value) const
+ {
+ while (!buf[place_value].isZero(*this))
+ {
+ place_value = grower.next(place_value);
+#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
+ ++collisions;
+#endif
+ }
+
+ return place_value;
+ }
+
+ void alloc(const Grower & new_grower)
+ {
+ buf = reinterpret_cast<Cell *>(Allocator::alloc(new_grower.bufSize() * sizeof(Cell)));
+ grower = new_grower;
+ }
+
+ void free()
+ {
+ if (buf)
+ {
+ Allocator::free(buf, getBufferSizeInBytes());
+ buf = nullptr;
+ }
+ }
+
+ /// Increase the size of the buffer.
+ void resize(size_t for_num_elems = 0, size_t for_buf_size = 0)
+ {
+#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
+ Stopwatch watch;
+#endif
+
+ size_t old_size = grower.bufSize();
+
+ /** In case of exception for the object to remain in the correct state,
+ * changing the variable `grower` (which determines the buffer size of the hash table)
+ * is postponed for a moment after a real buffer change.
+ * The temporary variable `new_grower` is used to determine the new size.
+ */
+ Grower new_grower = grower;
+
+ if (for_num_elems)
+ {
+ new_grower.set(for_num_elems);
+ if (new_grower.bufSize() <= old_size)
+ return;
+ }
+ else if (for_buf_size)
+ {
+ new_grower.setBufSize(for_buf_size);
+ if (new_grower.bufSize() <= old_size)
+ return;
+ }
+ else
+ new_grower.increaseSize();
+
+ /// Expand the space.
+
+ size_t old_buffer_size = getBufferSizeInBytes();
+
+ /** If cell required to be notified during move we need to temporary keep old buffer
+ * because realloc does not quarantee for reallocated buffer to have same base address
+ */
+ using Deleter = AllocatorBufferDeleter<Cell::need_to_notify_cell_during_move, Allocator, Cell>;
+ Deleter buffer_deleter(*this, old_buffer_size);
+ std::unique_ptr<Cell, Deleter> old_buffer(buf, buffer_deleter);
+
+ if constexpr (Cell::need_to_notify_cell_during_move)
+ {
+ buf = reinterpret_cast<Cell *>(Allocator::alloc(new_grower.bufSize() * sizeof(Cell)));
+ memcpy(reinterpret_cast<void *>(buf), reinterpret_cast<const void *>(old_buffer.get()), old_buffer_size);
+ }
+ else
+ buf = reinterpret_cast<Cell *>(Allocator::realloc(buf, old_buffer_size, new_grower.bufSize() * sizeof(Cell)));
+
+ grower = new_grower;
+
+ /** Now some items may need to be moved to a new location.
+ * The element can stay in place, or move to a new location "on the right",
+ * or move to the left of the collision resolution chain, because the elements to the left of it have been moved to the new "right" location.
+ */
+ size_t i = 0;
+ for (; i < old_size; ++i)
+ if (!buf[i].isZero(*this))
+ {
+ size_t updated_place_value = reinsert(buf[i], buf[i].getHash(*this));
+
+ if constexpr (Cell::need_to_notify_cell_during_move)
+ Cell::move(&(old_buffer.get())[i], &buf[updated_place_value]);
+ }
+
+ /** There is also a special case:
+ * if the element was to be at the end of the old buffer, [ x]
+ * but is at the beginning because of the collision resolution chain, [o x]
+ * then after resizing, it will first be out of place again, [ xo ]
+ * and in order to transfer it where necessary,
+ * after transferring all the elements from the old halves you need to [ o x ]
+ * process tail from the collision resolution chain immediately after it [ o x ]
+ */
+ size_t new_size = grower.bufSize();
+ for (; i < new_size && !buf[i].isZero(*this); ++i)
+ {
+ size_t updated_place_value = reinsert(buf[i], buf[i].getHash(*this));
+
+ if constexpr (Cell::need_to_notify_cell_during_move)
+ if (&buf[i] != &buf[updated_place_value])
+ Cell::move(&buf[i], &buf[updated_place_value]);
+ }
+
+#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
+ watch.stop();
+ std::cerr << std::fixed << std::setprecision(3)
+ << "Resize from " << old_size << " to " << grower.bufSize() << " took " << watch.elapsedSeconds() << " sec."
+ << std::endl;
+#endif
+ }
+
+
+ /** Paste into the new buffer the value that was in the old buffer.
+ * Used when increasing the buffer size.
+ */
+ size_t reinsert(Cell & x, size_t hash_value)
+ {
+ size_t place_value = grower.place(hash_value);
+
+ /// If the element is in its place.
+ if (&x == &buf[place_value])
+ return place_value;
+
+ /// Compute a new location, taking into account the collision resolution chain.
+ place_value = findCell(Cell::getKey(x.getValue()), hash_value, place_value);
+
+ /// If the item remains in its place in the old collision resolution chain.
+ if (!buf[place_value].isZero(*this))
+ return place_value;
+
+ /// Copy to a new location and zero the old one.
+ x.setHash(hash_value);
+ memcpy(static_cast<void*>(&buf[place_value]), &x, sizeof(x));
+ x.setZero();
+
+ /// Then the elements that previously were in collision with this can move to the old place.
+ return place_value;
+ }
+
+
+ void destroyElements()
+ {
+ if (!std::is_trivially_destructible_v<Cell>)
+ {
+ for (iterator it = begin(), it_end = end(); it != it_end; ++it)
+ {
+ it.ptr->~Cell();
+ /// In case of poison_in_dtor=1 it will be poisoned,
+ /// but it maybe used later, during iteration.
+ ///
+ /// NOTE, that technically this is UB [1], but OK for now.
+ ///
+ /// [1]: https://github.com/google/sanitizers/issues/854#issuecomment-329661378
+ //__msan_unpoison(it.ptr, sizeof(*it.ptr));
+ }
+ }
+ }
+
+
+ template <typename Derived, bool is_const>
+ class iterator_base
+ {
+ using Container = std::conditional_t<is_const, const Self, Self>;
+ using cell_type = std::conditional_t<is_const, const Cell, Cell>;
+
+ Container * container;
+ cell_type * ptr;
+
+ friend class HashTable;
+
+ public:
+ iterator_base() {}
+ iterator_base(Container * container_, cell_type * ptr_) : container(container_), ptr(ptr_) {}
+
+ bool operator== (const iterator_base & rhs) const { return ptr == rhs.ptr; }
+ bool operator!= (const iterator_base & rhs) const { return ptr != rhs.ptr; }
+
+ Derived & operator++()
+ {
+ /// If iterator was pointed to ZeroValueStorage, move it to the beginning of the main buffer.
+ if (unlikely(ptr->isZero(*container)))
+ ptr = container->buf;
+ else
+ ++ptr;
+
+ /// Skip empty cells in the main buffer.
+ auto buf_end = container->buf + container->grower.bufSize();
+ while (ptr < buf_end && ptr->isZero(*container))
+ ++ptr;
+
+ return static_cast<Derived &>(*this);
+ }
+
+ auto & operator* () const { return *ptr; }
+ auto * operator->() const { return ptr; }
+
+ auto getPtr() const { return ptr; }
+ size_t getHash() const { return ptr->getHash(*container); }
+
+ size_t getCollisionChainLength() const
+ {
+ return container->grower.place((ptr - container->buf) - container->grower.place(getHash()));
+ }
+
+ /**
+ * A hack for HashedDictionary.
+ *
+ * The problem: std-like find() returns an iterator, which has to be
+ * compared to end(). On the other hand, HashMap::find() returns
+ * LookupResult, which is compared to nullptr. HashedDictionary has to
+ * support both hash maps with the same code, hence the need for this
+ * hack.
+ *
+ * The proper way would be to remove iterator interface from our
+ * HashMap completely, change all its users to the existing internal
+ * iteration interface, and redefine end() to return LookupResult for
+ * compatibility with std find(). Unfortunately, now is not the time to
+ * do this.
+ */
+ operator Cell * () const { return nullptr; }
+ };
+
+
+public:
+ using key_type = Key;
+ using mapped_type = typename Cell::mapped_type;
+ using value_type = typename Cell::value_type;
+ using cell_type = Cell;
+
+ using LookupResult = Cell *;
+ using ConstLookupResult = const Cell *;
+
+ size_t hash(const Key & x) const { return Hash::operator()(x); }
+
+
+ HashTable()
+ {
+ if (Cell::need_zero_value_storage)
+ this->zeroValue()->setZero();
+ alloc(grower);
+ }
+
+ HashTable(size_t reserve_for_num_elements)
+ {
+ if (Cell::need_zero_value_storage)
+ this->zeroValue()->setZero();
+ grower.set(reserve_for_num_elements);
+ alloc(grower);
+ }
+
+ HashTable(HashTable && rhs)
+ : buf(nullptr)
+ {
+ *this = std::move(rhs);
+ }
+
+ ~HashTable()
+ {
+ destroyElements();
+ free();
+ }
+
+ HashTable & operator= (HashTable && rhs)
+ {
+ destroyElements();
+ free();
+
+ std::swap(buf, rhs.buf);
+ std::swap(m_size, rhs.m_size);
+ std::swap(grower, rhs.grower);
+
+ Hash::operator=(std::move(rhs));
+ Allocator::operator=(std::move(rhs));
+ Cell::State::operator=(std::move(rhs));
+ ZeroValueStorage<Cell::need_zero_value_storage, Cell>::operator=(std::move(rhs));
+
+ return *this;
+ }
+#if 0
+ class Reader final : private Cell::State
+ {
+ public:
+ Reader(DB::ReadBuffer & in_)
+ : in(in_)
+ {
+ }
+
+ Reader(const Reader &) = delete;
+ Reader & operator=(const Reader &) = delete;
+
+ bool next()
+ {
+ if (!is_initialized)
+ {
+ Cell::State::read(in);
+ DB::readVarUInt(size, in);
+ is_initialized = true;
+ }
+
+ if (read_count == size)
+ {
+ is_eof = true;
+ return false;
+ }
+
+ cell.read(in);
+ ++read_count;
+
+ return true;
+ }
+
+ inline const value_type & get() const
+ {
+ if (!is_initialized || is_eof)
+ throw DB::Exception("No available data");
+
+ return cell.getValue();
+ }
+
+ private:
+ DB::ReadBuffer & in;
+ Cell cell;
+ size_t read_count = 0;
+ size_t size = 0;
+ bool is_eof = false;
+ bool is_initialized = false;
+ };
+#endif
+
+ class iterator : public iterator_base<iterator, false>
+ {
+ public:
+ using iterator_base<iterator, false>::iterator_base;
+ };
+
+ class const_iterator : public iterator_base<const_iterator, true>
+ {
+ public:
+ using iterator_base<const_iterator, true>::iterator_base;
+ };
+
+
+ const_iterator begin() const
+ {
+ if (!buf)
+ return end();
+
+ if (this->hasZero())
+ return iteratorToZero();
+
+ const Cell * ptr = buf;
+ auto buf_end = buf + grower.bufSize();
+ while (ptr < buf_end && ptr->isZero(*this))
+ ++ptr;
+
+ return const_iterator(this, ptr);
+ }
+
+ const_iterator cbegin() const { return begin(); }
+
+ iterator begin()
+ {
+ if (!buf)
+ return end();
+
+ if (this->hasZero())
+ return iteratorToZero();
+
+ Cell * ptr = buf;
+ auto buf_end = buf + grower.bufSize();
+ while (ptr < buf_end && ptr->isZero(*this))
+ ++ptr;
+
+ return iterator(this, ptr);
+ }
+
+ const_iterator end() const
+ {
+ /// Avoid UBSan warning about adding zero to nullptr. It is valid in C++20 (and earlier) but not valid in C.
+ return const_iterator(this, buf ? buf + grower.bufSize() : buf);
+ }
+
+ const_iterator cend() const
+ {
+ return end();
+ }
+
+ iterator end()
+ {
+ return iterator(this, buf ? buf + grower.bufSize() : buf);
+ }
+
+
+protected:
+ const_iterator iteratorTo(const Cell * ptr) const { return const_iterator(this, ptr); }
+ iterator iteratorTo(Cell * ptr) { return iterator(this, ptr); }
+ const_iterator iteratorToZero() const { return iteratorTo(this->zeroValue()); }
+ iterator iteratorToZero() { return iteratorTo(this->zeroValue()); }
+
+
+ /// If the key is zero, insert it into a special place and return true.
+ /// We don't have to persist a zero key, because it's not actually inserted.
+ /// That's why we just take a Key by value, an not a key holder.
+ bool ALWAYS_INLINE emplaceIfZero(const Key & x, LookupResult & it, bool & inserted, size_t hash_value)
+ {
+ /// If it is claimed that the zero key can not be inserted into the table.
+ if (!Cell::need_zero_value_storage)
+ return false;
+
+ if (Cell::isZero(x, *this))
+ {
+ it = this->zeroValue();
+
+ if (!this->hasZero())
+ {
+ ++m_size;
+ this->setHasZero();
+ this->zeroValue()->setHash(hash_value);
+ inserted = true;
+ }
+ else
+ inserted = false;
+
+ return true;
+ }
+
+ return false;
+ }
+
+ template <typename KeyHolder>
+ void ALWAYS_INLINE emplaceNonZeroImpl(size_t place_value, KeyHolder && key_holder,
+ LookupResult & it, bool & inserted, size_t hash_value)
+ {
+ it = &buf[place_value];
+
+ if (!buf[place_value].isZero(*this))
+ {
+ keyHolderDiscardKey(key_holder);
+ inserted = false;
+ return;
+ }
+
+ keyHolderPersistKey(key_holder);
+ const auto & key = keyHolderGetKey(key_holder);
+
+ new (&buf[place_value]) Cell(key, *this);
+ buf[place_value].setHash(hash_value);
+ inserted = true;
+ ++m_size;
+
+ if (unlikely(grower.overflow(m_size)))
+ {
+ try
+ {
+ resize();
+ }
+ catch (...)
+ {
+ /** If we have not resized successfully, then there will be problems.
+ * There remains a key, but uninitialized mapped-value,
+ * which, perhaps, can not even be called a destructor.
+ */
+ --m_size;
+ buf[place_value].setZero();
+ inserted = false;
+ throw;
+ }
+
+ // The hash table was rehashed, so we have to re-find the key.
+ size_t new_place = findCell(key, hash_value, grower.place(hash_value));
+ assert(!buf[new_place].isZero(*this));
+ it = &buf[new_place];
+ }
+ }
+
+ /// Only for non-zero keys. Find the right place, insert the key there, if it does not already exist. Set iterator to the cell in output parameter.
+ template <typename KeyHolder>
+ void ALWAYS_INLINE emplaceNonZero(KeyHolder && key_holder, LookupResult & it,
+ bool & inserted, size_t hash_value)
+ {
+ const auto & key = keyHolderGetKey(key_holder);
+ size_t place_value = findCell(key, hash_value, grower.place(hash_value));
+ emplaceNonZeroImpl(place_value, key_holder, it, inserted, hash_value);
+ }
+
+
+public:
+ void reserve(size_t num_elements)
+ {
+ resize(num_elements);
+ }
+
+ /// Insert a value. In the case of any more complex values, it is better to use the `emplace` function.
+ std::pair<LookupResult, bool> ALWAYS_INLINE insert(const value_type & x)
+ {
+ std::pair<LookupResult, bool> res;
+
+ size_t hash_value = hash(Cell::getKey(x));
+ if (!emplaceIfZero(Cell::getKey(x), res.first, res.second, hash_value))
+ {
+ emplaceNonZero(Cell::getKey(x), res.first, res.second, hash_value);
+ }
+
+ if (res.second)
+ insertSetMapped(res.first->getMapped(), x);
+
+ return res;
+ }
+
+
+ /// Reinsert node pointed to by iterator
+ void ALWAYS_INLINE reinsert(iterator & it, size_t hash_value)
+ {
+ size_t place_value = reinsert(*it.getPtr(), hash_value);
+
+ if constexpr (Cell::need_to_notify_cell_during_move)
+ if (it.getPtr() != &buf[place_value])
+ Cell::move(it.getPtr(), &buf[place_value]);
+ }
+
+
+ /** Insert the key.
+ * Return values:
+ * 'it' -- a LookupResult pointing to the corresponding key/mapped pair.
+ * 'inserted' -- whether a new key was inserted.
+ *
+ * You have to make `placement new` of value if you inserted a new key,
+ * since when destroying a hash table, it will call the destructor!
+ *
+ * Example usage:
+ *
+ * Map::LookupResult it;
+ * bool inserted;
+ * map.emplace(key, it, inserted);
+ * if (inserted)
+ * new (&it->getMapped()) Mapped(value);
+ */
+ template <typename KeyHolder>
+ void ALWAYS_INLINE emplace(KeyHolder && key_holder, LookupResult & it, bool & inserted)
+ {
+ const auto & key = keyHolderGetKey(key_holder);
+ emplace(key_holder, it, inserted, hash(key));
+ }
+
+ template <typename KeyHolder>
+ void ALWAYS_INLINE emplace(KeyHolder && key_holder, LookupResult & it,
+ bool & inserted, size_t hash_value)
+ {
+ const auto & key = keyHolderGetKey(key_holder);
+ if (!emplaceIfZero(key, it, inserted, hash_value))
+ emplaceNonZero(key_holder, it, inserted, hash_value);
+ }
+
+ /// Copy the cell from another hash table. It is assumed that the cell is not zero, and also that there was no such key in the table yet.
+ void ALWAYS_INLINE insertUniqueNonZero(const Cell * cell, size_t hash_value)
+ {
+ size_t place_value = findEmptyCell(grower.place(hash_value));
+
+ memcpy(static_cast<void*>(&buf[place_value]), cell, sizeof(*cell));
+ ++m_size;
+
+ if (unlikely(grower.overflow(m_size)))
+ resize();
+ }
+
+ LookupResult ALWAYS_INLINE find(const Key & x)
+ {
+ if (Cell::isZero(x, *this))
+ return this->hasZero() ? this->zeroValue() : nullptr;
+
+ size_t hash_value = hash(x);
+ size_t place_value = findCell(x, hash_value, grower.place(hash_value));
+ return !buf[place_value].isZero(*this) ? &buf[place_value] : nullptr;
+ }
+
+ ConstLookupResult ALWAYS_INLINE find(const Key & x) const
+ {
+ return const_cast<std::decay_t<decltype(*this)> *>(this)->find(x);
+ }
+
+ LookupResult ALWAYS_INLINE find(const Key & x, size_t hash_value)
+ {
+ if (Cell::isZero(x, *this))
+ return this->hasZero() ? this->zeroValue() : nullptr;
+
+ size_t place_value = findCell(x, hash_value, grower.place(hash_value));
+ return !buf[place_value].isZero(*this) ? &buf[place_value] : nullptr;
+ }
+
+ ConstLookupResult ALWAYS_INLINE find(const Key & x, size_t hash_value) const
+ {
+ return const_cast<std::decay_t<decltype(*this)> *>(this)->find(x, hash_value);
+ }
+
+ std::enable_if_t<Grower::performs_linear_probing_with_single_step, bool>
+ ALWAYS_INLINE erase(const Key & x)
+ {
+ return erase(x, hash(x));
+ }
+
+ std::enable_if_t<Grower::performs_linear_probing_with_single_step, bool>
+ ALWAYS_INLINE erase(const Key & x, size_t hash_value)
+ {
+ /** Deletion from open addressing hash table without tombstones
+ *
+ * https://en.wikipedia.org/wiki/Linear_probing
+ * https://en.wikipedia.org/wiki/Open_addressing
+ * Algorithm without recomputing hash but keep probes difference value (difference of natural cell position and inserted one)
+ * in cell https://arxiv.org/ftp/arxiv/papers/0909/0909.2547.pdf
+ *
+ * Currently we use algorithm with hash recomputing on each step from https://en.wikipedia.org/wiki/Open_addressing
+ */
+
+ if (Cell::isZero(x, *this))
+ {
+ if (this->hasZero())
+ {
+ --m_size;
+ this->clearHasZero();
+ return true;
+ }
+ else
+ {
+ return false;
+ }
+ }
+
+ size_t erased_key_position = findCell(x, hash_value, grower.place(hash_value));
+
+ /// Key is not found
+ if (buf[erased_key_position].isZero(*this))
+ return false;
+
+ /// We need to guarantee loop termination because there will be empty position
+ assert(m_size < grower.bufSize());
+
+ size_t next_position = erased_key_position;
+
+ /**
+ * During element deletion there is a possibility that the search will be broken for one
+ * of the following elements, because this place erased_key_position is empty. We will check
+ * next_element. Consider a sequence from (erased_key_position, next_element], if the
+ * optimal_position of next_element falls into it, then removing erased_key_position
+ * will not break search for next_element.
+ * If optimal_position of the element does not fall into the sequence (erased_key_position, next_element]
+ * then deleting a erased_key_position will break search for it, so we need to move next_element
+ * to erased_key_position. Now we have empty place at next_element, so we apply the identical
+ * procedure for it.
+ * If an empty element is encountered then means that there is no more next elements for which we can
+ * break the search so we can exit.
+ */
+
+ /// Walk to the right through collision resolution chain and move elements to better positions
+ while (true)
+ {
+ next_position = grower.next(next_position);
+
+ /// If there's no more elements in the chain
+ if (buf[next_position].isZero(*this))
+ break;
+
+ /// The optimal position of the element in the cell at next_position
+ size_t optimal_position = grower.place(buf[next_position].getHash(*this));
+
+ /// If position of this element is already optimal - proceed to the next element.
+ if (optimal_position == next_position)
+ continue;
+
+ /// Cannot move this element because optimal position is after the freed place
+ /// The second condition is tricky - if the chain was overlapped before erased_key_position,
+ /// and the optimal position is actually before in collision resolution chain:
+ ///
+ /// [*xn***----------------***]
+ /// ^^-next elem ^
+ /// | |
+ /// erased elem the optimal position of the next elem
+ ///
+ /// so, the next elem should be moved to position of erased elem
+
+ /// The case of non overlapping part of chain
+ if (next_position > erased_key_position
+ && (optimal_position > erased_key_position) && (optimal_position < next_position))
+ {
+ continue;
+ }
+
+ /// The case of overlapping chain
+ if (next_position < erased_key_position
+ /// Cannot move this element because optimal position is after the freed place
+ && ((optimal_position > erased_key_position) || (optimal_position < next_position)))
+ {
+ continue;
+ }
+
+ /// Move the element to the freed place
+ memcpy(static_cast<void *>(&buf[erased_key_position]), static_cast<void *>(&buf[next_position]), sizeof(Cell));
+
+ if constexpr (Cell::need_to_notify_cell_during_move)
+ Cell::move(&buf[next_position], &buf[erased_key_position]);
+
+ /// Now we have another freed place
+ erased_key_position = next_position;
+ }
+
+ buf[erased_key_position].setZero();
+ --m_size;
+
+ return true;
+ }
+
+ bool ALWAYS_INLINE has(const Key & x) const
+ {
+ if (Cell::isZero(x, *this))
+ return this->hasZero();
+
+ size_t hash_value = hash(x);
+ size_t place_value = findCell(x, hash_value, grower.place(hash_value));
+ return !buf[place_value].isZero(*this);
+ }
+
+
+ bool ALWAYS_INLINE has(const Key & x, size_t hash_value) const
+ {
+ if (Cell::isZero(x, *this))
+ return this->hasZero();
+
+ size_t place_value = findCell(x, hash_value, grower.place(hash_value));
+ return !buf[place_value].isZero(*this);
+ }
+
+#if 0
+ void write(DB::WriteBuffer & wb) const
+ {
+ Cell::State::write(wb);
+ DB::writeVarUInt(m_size, wb);
+
+ if (this->hasZero())
+ this->zeroValue()->write(wb);
+
+ if (!buf)
+ return;
+
+ for (auto ptr = buf, buf_end = buf + grower.bufSize(); ptr < buf_end; ++ptr)
+ if (!ptr->isZero(*this))
+ ptr->write(wb);
+ }
+
+ void writeText(DB::WriteBuffer & wb) const
+ {
+ Cell::State::writeText(wb);
+ DB::writeText(m_size, wb);
+
+ if (this->hasZero())
+ {
+ DB::writeChar(',', wb);
+ this->zeroValue()->writeText(wb);
+ }
+
+ if (!buf)
+ return;
+
+ for (auto ptr = buf, buf_end = buf + grower.bufSize(); ptr < buf_end; ++ptr)
+ {
+ if (!ptr->isZero(*this))
+ {
+ DB::writeChar(',', wb);
+ ptr->writeText(wb);
+ }
+ }
+ }
+
+ void read(DB::ReadBuffer & rb)
+ {
+ Cell::State::read(rb);
+
+ destroyElements();
+ this->clearHasZero();
+ m_size = 0;
+
+ size_t new_size = 0;
+ DB::readVarUInt(new_size, rb);
+
+ free();
+ Grower new_grower = grower;
+ new_grower.set(new_size);
+ alloc(new_grower);
+
+ for (size_t i = 0; i < new_size; ++i)
+ {
+ Cell x;
+ x.read(rb);
+ insert(x.getValue());
+ }
+ }
+
+ void readText(DB::ReadBuffer & rb)
+ {
+ Cell::State::readText(rb);
+
+ destroyElements();
+ this->clearHasZero();
+ m_size = 0;
+
+ size_t new_size = 0;
+ DB::readText(new_size, rb);
+
+ free();
+ Grower new_grower = grower;
+ new_grower.set(new_size);
+ alloc(new_grower);
+
+ for (size_t i = 0; i < new_size; ++i)
+ {
+ Cell x;
+ DB::assertChar(',', rb);
+ x.readText(rb);
+ insert(x.getValue());
+ }
+ }
+#endif
+
+ size_t size() const
+ {
+ return m_size;
+ }
+
+ bool empty() const
+ {
+ return 0 == m_size;
+ }
+
+ void clear()
+ {
+ destroyElements();
+ this->clearHasZero();
+ m_size = 0;
+
+ memset(static_cast<void*>(buf), 0, grower.bufSize() * sizeof(*buf));
+ }
+
+ /// After executing this function, the table can only be destroyed,
+ /// and also you can use the methods `size`, `empty`, `begin`, `end`.
+ void clearAndShrink()
+ {
+ destroyElements();
+ this->clearHasZero();
+ m_size = 0;
+ free();
+ }
+
+ size_t getBufferSizeInBytes() const
+ {
+ return grower.bufSize() * sizeof(Cell);
+ }
+
+ size_t getBufferSizeInCells() const
+ {
+ return grower.bufSize();
+ }
+
+ /// Return offset for result in internal buffer.
+ /// Result can have value up to `getBufferSizeInCells() + 1`
+ /// because offset for zero value considered to be 0
+ /// and for other values it will be `offset in buffer + 1`
+ size_t offsetInternal(ConstLookupResult ptr) const
+ {
+ if (ptr->isZero(*this))
+ return 0;
+ return ptr - buf + 1;
+ }
+
+#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
+ size_t getCollisions() const
+ {
+ return collisions;
+ }
+#endif
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/HashTableAllocator.h b/ydb/library/arrow_clickhouse/Common/HashTable/HashTableAllocator.h
new file mode 100644
index 00000000000..6522a5e27aa
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/HashTableAllocator.h
@@ -0,0 +1,22 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <Common/Allocator.h>
+
+namespace CH
+{
+
+/**
+ * We are going to use the entire memory we allocated when resizing a hash
+ * table, so it makes sense to pre-fault the pages so that page faults don't
+ * interrupt the resize loop. Set the allocator parameter accordingly.
+ */
+using HashTableAllocator = Allocator<true /* clear_memory */, true /* mmap_populate */>;
+
+template <size_t initial_bytes = 64>
+using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory<HashTableAllocator, initial_bytes>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/HashTableKeyHolder.h b/ydb/library/arrow_clickhouse/Common/HashTable/HashTableKeyHolder.h
new file mode 100644
index 00000000000..e55400103da
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/HashTableKeyHolder.h
@@ -0,0 +1,127 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <Common/Arena.h>
+
+/**
+ * In some aggregation scenarios, when adding a key to the hash table, we
+ * start with a temporary key object, and if it turns out to be a new key,
+ * we must make it persistent (e.g. copy to an Arena) and use the resulting
+ * persistent object as hash table key. This happens only for StringRef keys,
+ * because other key types are stored by value, but StringRef is a pointer-like
+ * type: the actual data are stored elsewhere. Even for StringRef, we don't
+ * make a persistent copy of the key in each of the following cases:
+ * 1) the aggregation method doesn't use temporary keys, so they're persistent
+ * from the start;
+ * 1) the key is already present in the hash table;
+ * 3) that particular key is stored by value, e.g. a short StringRef key in
+ * StringHashMap.
+ *
+ * In the past, the caller was responsible for making the key persistent after
+ * in was inserted. emplace() returned whether the key is new or not, so the
+ * caller only stored new keys (this is case (2) from the above list). However,
+ * now we are adding a compound hash table for StringRef keys, so case (3)
+ * appears. The decision about persistence now depends on some properties of
+ * the key, and the logic of this decision is tied to the particular hash table
+ * implementation. This means that the hash table user now doesn't have enough
+ * data and logic to make this decision by itself.
+ *
+ * To support these new requirements, we now manage key persistence by passing
+ * a special key holder to emplace(), which has the functions to make the key
+ * persistent or to discard it. emplace() then calls these functions at the
+ * appropriate moments.
+ *
+ * This approach has the following benefits:
+ * - no extra runtime branches in the caller to make the key persistent.
+ * - no additional data is stored in the hash table itself, which is important
+ * when it's used in aggregate function states.
+ * - no overhead when the key memory management isn't needed: we just pass the
+ * bare key without any wrapper to emplace(), and the default callbacks do
+ * nothing.
+ *
+ * This file defines the default key persistence functions, as well as two
+ * different key holders and corresponding functions for storing StringRef
+ * keys to Arena.
+ */
+
+namespace CH
+{
+
+/**
+ * Returns the key. Can return the temporary key initially.
+ * After the call to keyHolderPersistKey(), must return the persistent key.
+ */
+template <typename Key>
+inline Key & ALWAYS_INLINE keyHolderGetKey(Key && key) { return key; }
+
+/**
+ * Make the key persistent. keyHolderGetKey() must return the persistent key
+ * after this call.
+ */
+template <typename Key>
+inline void ALWAYS_INLINE keyHolderPersistKey(Key &&) {}
+
+/**
+ * Discard the key. Calling keyHolderGetKey() is ill-defined after this.
+ */
+template <typename Key>
+inline void ALWAYS_INLINE keyHolderDiscardKey(Key &&) {}
+
+/**
+ * ArenaKeyHolder is a key holder for hash tables that serializes a StringRef
+ * key to an Arena.
+ */
+struct ArenaKeyHolder
+{
+ StringRef key;
+ Arena & pool;
+
+};
+
+inline StringRef & ALWAYS_INLINE keyHolderGetKey(CH::ArenaKeyHolder & holder)
+{
+ return holder.key;
+}
+
+inline void ALWAYS_INLINE keyHolderPersistKey(CH::ArenaKeyHolder & holder)
+{
+ // Hash table shouldn't ask us to persist a zero key
+ assert(holder.key.size > 0);
+ holder.key.data = holder.pool.insert(holder.key.data, holder.key.size);
+}
+
+inline void ALWAYS_INLINE keyHolderDiscardKey(CH::ArenaKeyHolder &)
+{
+}
+
+/** SerializedKeyHolder is a key holder for a StringRef key that is already
+ * serialized to an Arena. The key must be the last allocation in this Arena,
+ * and is discarded by rolling back the allocation.
+ */
+struct SerializedKeyHolder
+{
+ StringRef key;
+ Arena & pool;
+};
+
+inline StringRef & ALWAYS_INLINE keyHolderGetKey(CH::SerializedKeyHolder & holder)
+{
+ return holder.key;
+}
+
+inline void ALWAYS_INLINE keyHolderPersistKey(CH::SerializedKeyHolder &)
+{
+}
+
+inline void ALWAYS_INLINE keyHolderDiscardKey(CH::SerializedKeyHolder & holder)
+{
+ [[maybe_unused]] void * new_head = holder.pool.rollback(holder.key.size);
+ assert(new_head == holder.key.data);
+ holder.key.data = nullptr;
+ holder.key.size = 0;
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/StringHashMap.h b/ydb/library/arrow_clickhouse/Common/HashTable/StringHashMap.h
new file mode 100644
index 00000000000..bb6e0994c3b
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/StringHashMap.h
@@ -0,0 +1,198 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <Common/HashTable/HashMap.h>
+#include <Common/HashTable/HashTableAllocator.h>
+#include <Common/HashTable/StringHashTable.h>
+
+namespace CH
+{
+
+template <typename Key, typename TMapped>
+struct StringHashMapCell : public HashMapCell<Key, TMapped, StringHashTableHash, HashTableNoState>
+{
+ using Base = HashMapCell<Key, TMapped, StringHashTableHash, HashTableNoState>;
+ using value_type = typename Base::value_type;
+ using Base::Base;
+ static constexpr bool need_zero_value_storage = false;
+ // external
+ const StringRef getKey() const { return toStringRef(this->value.first); }
+ // internal
+ static const Key & getKey(const value_type & value_) { return value_.first; }
+};
+
+template <typename TMapped>
+struct StringHashMapCell<StringKey16, TMapped> : public HashMapCell<StringKey16, TMapped, StringHashTableHash, HashTableNoState>
+{
+ using Base = HashMapCell<StringKey16, TMapped, StringHashTableHash, HashTableNoState>;
+ using value_type = typename Base::value_type;
+ using Base::Base;
+ static constexpr bool need_zero_value_storage = false;
+ bool isZero(const HashTableNoState & state) const { return isZero(this->value.first, state); }
+
+ // Zero means unoccupied cells in hash table. Use key with last word = 0 as
+ // zero keys, because such keys are unrepresentable (no way to encode length).
+ static bool isZero(const StringKey16 & key, const HashTableNoState &) { return key.items[1] == 0; }
+ void setZero() { this->value.first.items[1] = 0; }
+
+ // external
+ const StringRef getKey() const { return toStringRef(this->value.first); }
+ // internal
+ static const StringKey16 & getKey(const value_type & value_) { return value_.first; }
+};
+
+template <typename TMapped>
+struct StringHashMapCell<StringKey24, TMapped> : public HashMapCell<StringKey24, TMapped, StringHashTableHash, HashTableNoState>
+{
+ using Base = HashMapCell<StringKey24, TMapped, StringHashTableHash, HashTableNoState>;
+ using value_type = typename Base::value_type;
+ using Base::Base;
+ static constexpr bool need_zero_value_storage = false;
+ bool isZero(const HashTableNoState & state) const { return isZero(this->value.first, state); }
+
+ // Zero means unoccupied cells in hash table. Use key with last word = 0 as
+ // zero keys, because such keys are unrepresentable (no way to encode length).
+ static bool isZero(const StringKey24 & key, const HashTableNoState &)
+ { return key.c == 0; }
+ void setZero() { this->value.first.c = 0; }
+
+ // external
+ const StringRef getKey() const { return toStringRef(this->value.first); }
+ // internal
+ static const StringKey24 & getKey(const value_type & value_) { return value_.first; }
+};
+
+template <typename TMapped>
+struct StringHashMapCell<StringRef, TMapped> : public HashMapCellWithSavedHash<StringRef, TMapped, StringHashTableHash, HashTableNoState>
+{
+ using Base = HashMapCellWithSavedHash<StringRef, TMapped, StringHashTableHash, HashTableNoState>;
+ using value_type = typename Base::value_type;
+ using Base::Base;
+ static constexpr bool need_zero_value_storage = false;
+ // external
+ using Base::getKey;
+ // internal
+ static const StringRef & getKey(const value_type & value_) { return value_.first; }
+};
+
+template <typename TMapped, typename Allocator>
+struct StringHashMapSubMaps
+{
+ using T0 = StringHashTableEmpty<StringHashMapCell<StringRef, TMapped>>;
+ using T1 = HashMapTable<StringKey8, StringHashMapCell<StringKey8, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
+ using T2 = HashMapTable<StringKey16, StringHashMapCell<StringKey16, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
+ using T3 = HashMapTable<StringKey24, StringHashMapCell<StringKey24, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
+ using Ts = HashMapTable<StringRef, StringHashMapCell<StringRef, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
+};
+
+template <typename TMapped, typename Allocator = HashTableAllocator>
+class StringHashMap : public StringHashTable<StringHashMapSubMaps<TMapped, Allocator>>
+{
+public:
+ using Key = StringRef;
+ using Base = StringHashTable<StringHashMapSubMaps<TMapped, Allocator>>;
+ using Self = StringHashMap;
+ using LookupResult = typename Base::LookupResult;
+
+ using Base::Base;
+
+ /// Merge every cell's value of current map into the destination map.
+ /// Func should have signature void(Mapped & dst, Mapped & src, bool emplaced).
+ /// Each filled cell in current map will invoke func once. If that map doesn't
+ /// have a key equals to the given cell, a new cell gets emplaced into that map,
+ /// and func is invoked with the third argument emplaced set to true. Otherwise
+ /// emplaced is set to false.
+ template <typename Func>
+ void ALWAYS_INLINE mergeToViaEmplace(Self & that, Func && func)
+ {
+ if (this->m0.hasZero() && that.m0.hasZero())
+ func(that.m0.zeroValue()->getMapped(), this->m0.zeroValue()->getMapped(), false);
+ else if (this->m0.hasZero())
+ {
+ that.m0.setHasZero();
+ func(that.m0.zeroValue()->getMapped(), this->m0.zeroValue()->getMapped(), true);
+ }
+ this->m1.mergeToViaEmplace(that.m1, func);
+ this->m2.mergeToViaEmplace(that.m2, func);
+ this->m3.mergeToViaEmplace(that.m3, func);
+ this->ms.mergeToViaEmplace(that.ms, func);
+ }
+
+ /// Merge every cell's value of current map into the destination map via find.
+ /// Func should have signature void(Mapped & dst, Mapped & src, bool exist).
+ /// Each filled cell in current map will invoke func once. If that map doesn't
+ /// have a key equals to the given cell, func is invoked with the third argument
+ /// exist set to false. Otherwise exist is set to true.
+ template <typename Func>
+ void ALWAYS_INLINE mergeToViaFind(Self & that, Func && func)
+ {
+ if (this->m0.size() && that.m0.size())
+ func(that.m0.zeroValue()->getMapped(), this->m0.zeroValue()->getMapped(), true);
+ else if (this->m0.size())
+ func(this->m0.zeroValue()->getMapped(), this->m0.zeroValue()->getMapped(), false);
+ this->m1.mergeToViaFind(that.m1, func);
+ this->m2.mergeToViaFind(that.m2, func);
+ this->m3.mergeToViaFind(that.m3, func);
+ this->ms.mergeToViaFind(that.ms, func);
+ }
+
+ TMapped & ALWAYS_INLINE operator[](const Key & x)
+ {
+ LookupResult it;
+ bool inserted;
+ this->emplace(x, it, inserted);
+ if (inserted)
+ new (&it->getMapped()) TMapped();
+
+ return it->getMapped();
+ }
+
+ template <typename Func>
+ void ALWAYS_INLINE forEachValue(Func && func)
+ {
+ if (this->m0.size())
+ {
+ func(StringRef{}, this->m0.zeroValue()->getMapped());
+ }
+
+ for (auto & v : this->m1)
+ {
+ func(v.getKey(), v.getMapped());
+ }
+
+ for (auto & v : this->m2)
+ {
+ func(v.getKey(), v.getMapped());
+ }
+
+ for (auto & v : this->m3)
+ {
+ func(v.getKey(), v.getMapped());
+ }
+
+ for (auto & v : this->ms)
+ {
+ func(v.getKey(), v.getMapped());
+ }
+ }
+
+ template <typename Func>
+ void ALWAYS_INLINE forEachMapped(Func && func)
+ {
+ if (this->m0.size())
+ func(this->m0.zeroValue()->getMapped());
+ for (auto & v : this->m1)
+ func(v.getMapped());
+ for (auto & v : this->m2)
+ func(v.getMapped());
+ for (auto & v : this->m3)
+ func(v.getMapped());
+ for (auto & v : this->ms)
+ func(v.getMapped());
+ }
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/HashTable/StringHashTable.h b/ydb/library/arrow_clickhouse/Common/HashTable/StringHashTable.h
new file mode 100644
index 00000000000..da817544822
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/HashTable/StringHashTable.h
@@ -0,0 +1,444 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <Common/HashTable/HashMap.h>
+#include <Common/HashTable/HashTable.h>
+
+#include <new>
+#include <variant>
+
+namespace CH
+{
+
+using StringKey8 = UInt64;
+using StringKey16 = UInt128;
+struct StringKey24
+{
+ UInt64 a;
+ UInt64 b;
+ UInt64 c;
+
+ bool operator==(const StringKey24 rhs) const { return a == rhs.a && b == rhs.b && c == rhs.c; }
+};
+
+inline StringRef ALWAYS_INLINE toStringRef(const StringKey8 & n)
+{
+ assert(n != 0);
+ return {reinterpret_cast<const char *>(&n), 8ul - (__builtin_clzll(n) >> 3)};
+}
+inline StringRef ALWAYS_INLINE toStringRef(const StringKey16 & n)
+{
+ assert(n.items[1] != 0);
+ return {reinterpret_cast<const char *>(&n), 16ul - (__builtin_clzll(n.items[1]) >> 3)};
+}
+inline StringRef ALWAYS_INLINE toStringRef(const StringKey24 & n)
+{
+ assert(n.c != 0);
+ return {reinterpret_cast<const char *>(&n), 24ul - (__builtin_clzll(n.c) >> 3)};
+}
+
+struct StringHashTableHash
+{
+#if defined(__SSE4_2__)
+ size_t ALWAYS_INLINE operator()(StringKey8 key) const
+ {
+ size_t res = -1ULL;
+ res = _mm_crc32_u64(res, key);
+ return res;
+ }
+ size_t ALWAYS_INLINE operator()(StringKey16 key) const
+ {
+ size_t res = -1ULL;
+ res = _mm_crc32_u64(res, key.items[0]);
+ res = _mm_crc32_u64(res, key.items[1]);
+ return res;
+ }
+ size_t ALWAYS_INLINE operator()(StringKey24 key) const
+ {
+ size_t res = -1ULL;
+ res = _mm_crc32_u64(res, key.a);
+ res = _mm_crc32_u64(res, key.b);
+ res = _mm_crc32_u64(res, key.c);
+ return res;
+ }
+#else
+ size_t ALWAYS_INLINE operator()(StringKey8 key) const
+ {
+ return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 8);
+ }
+ size_t ALWAYS_INLINE operator()(StringKey16 key) const
+ {
+ return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 16);
+ }
+ size_t ALWAYS_INLINE operator()(StringKey24 key) const
+ {
+ return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 24);
+ }
+#endif
+ size_t ALWAYS_INLINE operator()(StringRef key) const
+ {
+ return StringRefHash()(key);
+ }
+};
+
+template <typename Cell>
+struct StringHashTableEmpty //-V730
+{
+ using Self = StringHashTableEmpty;
+
+ bool has_zero = false;
+ std::aligned_storage_t<sizeof(Cell), alignof(Cell)> zero_value_storage; /// Storage of element with zero key.
+
+public:
+ bool hasZero() const { return has_zero; }
+
+ void setHasZero()
+ {
+ has_zero = true;
+ new (zeroValue()) Cell();
+ }
+
+ void setHasZero(const Cell & other)
+ {
+ has_zero = true;
+ new (zeroValue()) Cell(other);
+ }
+
+ void clearHasZero()
+ {
+ has_zero = false;
+ if (!std::is_trivially_destructible_v<Cell>)
+ zeroValue()->~Cell();
+ }
+
+ Cell * zeroValue() { return std::launder(reinterpret_cast<Cell *>(&zero_value_storage)); }
+ const Cell * zeroValue() const { return std::launder(reinterpret_cast<const Cell *>(&zero_value_storage)); }
+
+ using LookupResult = Cell *;
+ using ConstLookupResult = const Cell *;
+
+ template <typename KeyHolder>
+ void ALWAYS_INLINE emplace(KeyHolder &&, LookupResult & it, bool & inserted, size_t = 0)
+ {
+ if (!hasZero())
+ {
+ setHasZero();
+ inserted = true;
+ }
+ else
+ inserted = false;
+ it = zeroValue();
+ }
+
+ template <typename Key>
+ LookupResult ALWAYS_INLINE find(const Key &, size_t = 0)
+ {
+ return hasZero() ? zeroValue() : nullptr;
+ }
+
+ template <typename Key>
+ ConstLookupResult ALWAYS_INLINE find(const Key &, size_t = 0) const
+ {
+ return hasZero() ? zeroValue() : nullptr;
+ }
+#if 0
+ void write(DB::WriteBuffer & wb) const { zeroValue()->write(wb); }
+ void writeText(DB::WriteBuffer & wb) const { zeroValue()->writeText(wb); }
+ void read(DB::ReadBuffer & rb) { zeroValue()->read(rb); }
+ void readText(DB::ReadBuffer & rb) { zeroValue()->readText(rb); }
+#endif
+ size_t size() const { return hasZero() ? 1 : 0; }
+ bool empty() const { return !hasZero(); }
+ size_t getBufferSizeInBytes() const { return sizeof(Cell); }
+ size_t getCollisions() const { return 0; }
+};
+
+template <size_t initial_size_degree = 8>
+struct StringHashTableGrower : public HashTableGrower<initial_size_degree>
+{
+ // Smooth growing for string maps
+ void increaseSize() { this->size_degree += 1; }
+};
+
+template <typename Mapped>
+struct StringHashTableLookupResult
+{
+ Mapped * mapped_ptr;
+ StringHashTableLookupResult() {}
+ StringHashTableLookupResult(Mapped * mapped_ptr_) : mapped_ptr(mapped_ptr_) {}
+ StringHashTableLookupResult(std::nullptr_t) {}
+ const VoidKey getKey() const { return {}; }
+ auto & getMapped() { return *mapped_ptr; }
+ auto & operator*() { return *this; }
+ auto & operator*() const { return *this; }
+ auto * operator->() { return this; }
+ auto * operator->() const { return this; }
+ operator bool() const { return mapped_ptr; }
+ friend bool operator==(const StringHashTableLookupResult & a, const nullptr_t &) { return !a.mapped_ptr; }
+ friend bool operator==(const std::nullptr_t &, const StringHashTableLookupResult & b) { return !b.mapped_ptr; }
+ friend bool operator!=(const StringHashTableLookupResult & a, const nullptr_t &) { return a.mapped_ptr; }
+ friend bool operator!=(const std::nullptr_t &, const StringHashTableLookupResult & b) { return b.mapped_ptr; }
+};
+
+template <typename SubMaps>
+class StringHashTable //: private boost::noncopyable
+{
+protected:
+ static constexpr size_t NUM_MAPS = 5;
+ // Map for storing empty string
+ using T0 = typename SubMaps::T0;
+
+ // Short strings are stored as numbers
+ using T1 = typename SubMaps::T1;
+ using T2 = typename SubMaps::T2;
+ using T3 = typename SubMaps::T3;
+
+ // Long strings are stored as StringRef along with saved hash
+ using Ts = typename SubMaps::Ts;
+ using Self = StringHashTable;
+
+ template <typename, typename, size_t>
+ friend class TwoLevelStringHashTable;
+
+ T0 m0;
+ T1 m1;
+ T2 m2;
+ T3 m3;
+ Ts ms;
+
+public:
+ using Key = StringRef;
+ using key_type = Key;
+ using mapped_type = typename Ts::mapped_type;
+ using value_type = typename Ts::value_type;
+ using cell_type = typename Ts::cell_type;
+
+ using LookupResult = StringHashTableLookupResult<typename cell_type::mapped_type>;
+ using ConstLookupResult = StringHashTableLookupResult<const typename cell_type::mapped_type>;
+
+ StringHashTable() = default;
+
+ StringHashTable(size_t reserve_for_num_elements)
+ : m1{reserve_for_num_elements / 4}
+ , m2{reserve_for_num_elements / 4}
+ , m3{reserve_for_num_elements / 4}
+ , ms{reserve_for_num_elements / 4}
+ {
+ }
+
+ StringHashTable(StringHashTable && rhs)
+ : m1(std::move(rhs.m1))
+ , m2(std::move(rhs.m2))
+ , m3(std::move(rhs.m3))
+ , ms(std::move(rhs.ms))
+ {
+ }
+
+ ~StringHashTable() = default;
+
+public:
+ // Dispatch is written in a way that maximizes the performance:
+ // 1. Always memcpy 8 times bytes
+ // 2. Use switch case extension to generate fast dispatching table
+ // 3. Funcs are named callables that can be force_inlined
+ //
+ // NOTE: It relies on Little Endianness
+ //
+ // NOTE: It requires padded to 8 bytes keys (IOW you cannot pass
+ // std::string here, but you can pass i.e. ColumnString::getDataAt()),
+ // since it copies 8 bytes at a time.
+ template <typename Self, typename KeyHolder, typename Func>
+ static auto ALWAYS_INLINE dispatch(Self & self, KeyHolder && key_holder, Func && func)
+ {
+ StringHashTableHash hash;
+ const StringRef & x = keyHolderGetKey(key_holder);
+ const size_t sz = x.size;
+ if (sz == 0)
+ {
+ keyHolderDiscardKey(key_holder);
+ return func(self.m0, VoidKey{}, 0);
+ }
+
+ if (x.data[sz - 1] == 0)
+ {
+ // Strings with trailing zeros are not representable as fixed-size
+ // string keys. Put them to the generic table.
+ return func(self.ms, std::forward<KeyHolder>(key_holder), hash(x));
+ }
+
+ const char * p = x.data;
+ // pending bits that needs to be shifted out
+ const char s = (-sz & 7) * 8;
+ union
+ {
+ StringKey8 k8;
+ StringKey16 k16;
+ StringKey24 k24;
+ UInt64 n[3];
+ };
+ switch ((sz - 1) >> 3)
+ {
+ case 0: // 1..8 bytes
+ {
+ // first half page
+ if ((reinterpret_cast<uintptr_t>(p) & 2048) == 0)
+ {
+ memcpy(&n[0], p, 8);
+ n[0] &= -1ul >> s;
+ }
+ else
+ {
+ const char * lp = x.data + x.size - 8;
+ memcpy(&n[0], lp, 8);
+ n[0] >>= s;
+ }
+ keyHolderDiscardKey(key_holder);
+ return func(self.m1, k8, hash(k8));
+ }
+ case 1: // 9..16 bytes
+ {
+ memcpy(&n[0], p, 8);
+ const char * lp = x.data + x.size - 8;
+ memcpy(&n[1], lp, 8);
+ n[1] >>= s;
+ keyHolderDiscardKey(key_holder);
+ return func(self.m2, k16, hash(k16));
+ }
+ case 2: // 17..24 bytes
+ {
+ memcpy(&n[0], p, 16);
+ const char * lp = x.data + x.size - 8;
+ memcpy(&n[2], lp, 8);
+ n[2] >>= s;
+ keyHolderDiscardKey(key_holder);
+ return func(self.m3, k24, hash(k24));
+ }
+ default: // >= 25 bytes
+ {
+ return func(self.ms, std::forward<KeyHolder>(key_holder), hash(x));
+ }
+ }
+ }
+
+ struct EmplaceCallable
+ {
+ LookupResult & mapped;
+ bool & inserted;
+
+ EmplaceCallable(LookupResult & mapped_, bool & inserted_)
+ : mapped(mapped_), inserted(inserted_) {}
+
+ template <typename Map, typename KeyHolder>
+ void ALWAYS_INLINE operator()(Map & map, KeyHolder && key_holder, size_t hash)
+ {
+ typename Map::LookupResult result;
+ map.emplace(key_holder, result, inserted, hash);
+ mapped = &result->getMapped();
+ }
+ };
+
+ template <typename KeyHolder>
+ void ALWAYS_INLINE emplace(KeyHolder && key_holder, LookupResult & it, bool & inserted)
+ {
+ this->dispatch(*this, key_holder, EmplaceCallable(it, inserted));
+ }
+
+ struct FindCallable
+ {
+ // find() doesn't need any key memory management, so we don't work with
+ // any key holders here, only with normal keys. The key type is still
+ // different for every subtable, this is why it is a template parameter.
+ template <typename Submap, typename SubmapKey>
+ auto ALWAYS_INLINE operator()(Submap & map, const SubmapKey & key, size_t hash)
+ {
+ auto it = map.find(key, hash);
+ if (!it)
+ return decltype(&it->getMapped()){};
+ else
+ return &it->getMapped();
+ }
+ };
+
+ LookupResult ALWAYS_INLINE find(const Key & x)
+ {
+ return dispatch(*this, x, FindCallable{});
+ }
+
+ ConstLookupResult ALWAYS_INLINE find(const Key & x) const
+ {
+ return dispatch(*this, x, FindCallable{});
+ }
+
+ bool ALWAYS_INLINE has(const Key & x, size_t = 0) const
+ {
+ return dispatch(*this, x, FindCallable{}) != nullptr;
+ }
+#if 0
+ void write(DB::WriteBuffer & wb) const
+ {
+ m0.write(wb);
+ m1.write(wb);
+ m2.write(wb);
+ m3.write(wb);
+ ms.write(wb);
+ }
+
+ void writeText(DB::WriteBuffer & wb) const
+ {
+ m0.writeText(wb);
+ DB::writeChar(',', wb);
+ m1.writeText(wb);
+ DB::writeChar(',', wb);
+ m2.writeText(wb);
+ DB::writeChar(',', wb);
+ m3.writeText(wb);
+ DB::writeChar(',', wb);
+ ms.writeText(wb);
+ }
+
+ void read(DB::ReadBuffer & rb)
+ {
+ m0.read(rb);
+ m1.read(rb);
+ m2.read(rb);
+ m3.read(rb);
+ ms.read(rb);
+ }
+
+ void readText(DB::ReadBuffer & rb)
+ {
+ m0.readText(rb);
+ DB::assertChar(',', rb);
+ m1.readText(rb);
+ DB::assertChar(',', rb);
+ m2.readText(rb);
+ DB::assertChar(',', rb);
+ m3.readText(rb);
+ DB::assertChar(',', rb);
+ ms.readText(rb);
+ }
+#endif
+ size_t size() const { return m0.size() + m1.size() + m2.size() + m3.size() + ms.size(); }
+
+ bool empty() const { return m0.empty() && m1.empty() && m2.empty() && m3.empty() && ms.empty(); }
+
+ size_t getBufferSizeInBytes() const
+ {
+ return m0.getBufferSizeInBytes() + m1.getBufferSizeInBytes() + m2.getBufferSizeInBytes() + m3.getBufferSizeInBytes()
+ + ms.getBufferSizeInBytes();
+ }
+
+ void clearAndShrink()
+ {
+ m1.clearHasZero();
+ m1.clearAndShrink();
+ m2.clearAndShrink();
+ m3.clearAndShrink();
+ ms.clearAndShrink();
+ }
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/PODArray.cpp b/ydb/library/arrow_clickhouse/Common/PODArray.cpp
new file mode 100644
index 00000000000..0840d8b7a01
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/PODArray.cpp
@@ -0,0 +1,23 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#include <Common/PODArray.h>
+
+namespace CH
+{
+
+/// Used for left padding of PODArray when empty
+const char empty_pod_array[empty_pod_array_size]{};
+
+template class PODArray<UInt8, 4096, Allocator<false>, 15, 16>;
+template class PODArray<UInt16, 4096, Allocator<false>, 15, 16>;
+template class PODArray<UInt32, 4096, Allocator<false>, 15, 16>;
+template class PODArray<UInt64, 4096, Allocator<false>, 15, 16>;
+
+template class PODArray<Int8, 4096, Allocator<false>, 15, 16>;
+template class PODArray<Int16, 4096, Allocator<false>, 15, 16>;
+template class PODArray<Int32, 4096, Allocator<false>, 15, 16>;
+template class PODArray<Int64, 4096, Allocator<false>, 15, 16>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/PODArray.h b/ydb/library/arrow_clickhouse/Common/PODArray.h
new file mode 100644
index 00000000000..45924b02225
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/PODArray.h
@@ -0,0 +1,803 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <string.h>
+#include <cstddef>
+#include <cassert>
+#include <algorithm>
+#include <memory>
+
+//#include <boost/noncopyable.hpp>
+
+#include <common/strong_typedef.h>
+
+#include <Common/Allocator.h>
+#include <Common/memcpySmall.h>
+
+#ifndef NDEBUG
+ #include <sys/mman.h>
+#endif
+
+#include <Common/PODArray_fwd.h>
+
+namespace CH
+{
+
+/** Whether we can use memcpy instead of a loop with assignment to T from U.
+ * It is Ok if types are the same. And if types are integral and of the same size,
+ * example: char, signed char, unsigned char.
+ * It's not Ok for int and float.
+ * Don't forget to apply std::decay when using this constexpr.
+ */
+template <typename T, typename U>
+constexpr bool memcpy_can_be_used_for_assignment = std::is_same_v<T, U>
+ || (std::is_integral_v<T> && std::is_integral_v<U> && sizeof(T) == sizeof(U));
+
+
+/** A dynamic array for POD types.
+ * Designed for a small number of large arrays (rather than a lot of small ones).
+ * To be more precise - for use in ColumnVector.
+ * It differs from std::vector in that it does not initialize the elements.
+ *
+ * Made noncopyable so that there are no accidental copies. You can copy the data using `assign` method.
+ *
+ * Only part of the std::vector interface is supported.
+ *
+ * The default constructor creates an empty object that does not allocate memory.
+ * Then the memory is allocated at least initial_bytes bytes.
+ *
+ * If you insert elements with push_back, without making a `reserve`, then PODArray is about 2.5 times faster than std::vector.
+ *
+ * The template parameter `pad_right` - always allocate at the end of the array as many unused bytes.
+ * Can be used to make optimistic reading, writing, copying with unaligned SIMD instructions.
+ *
+ * The template parameter `pad_left` - always allocate memory before 0th element of the array (rounded up to the whole number of elements)
+ * and zero initialize -1th element. It allows to use -1th element that will have value 0.
+ * This gives performance benefits when converting an array of offsets to array of sizes.
+ *
+ * Some methods using allocator have TAllocatorParams variadic arguments.
+ * These arguments will be passed to corresponding methods of TAllocator.
+ * Example: pointer to Arena, that is used for allocations.
+ *
+ * Why Allocator is not passed through constructor, as it is done in C++ standard library?
+ * Because sometimes we have many small objects, that share same allocator with same parameters,
+ * and we must avoid larger object size due to storing the same parameters in each object.
+ * This is required for states of aggregate functions.
+ *
+ * TODO Pass alignment to Allocator.
+ * TODO Allow greater alignment than alignof(T). Example: array of char aligned to page size.
+ */
+static constexpr size_t empty_pod_array_size = 1024;
+extern const char empty_pod_array[empty_pod_array_size];
+
+/** Base class that depend only on size of element, not on element itself.
+ * You can static_cast to this class if you want to insert some data regardless to the actual type T.
+ */
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wnull-dereference"
+
+inline size_t roundUpToPowerOfTwoOrZero(size_t n)
+{
+ // if MSB is set, return n, to avoid return zero
+ if (unlikely(n >= 0x8000000000000000ULL))
+ return n;
+
+ --n;
+ n |= n >> 1;
+ n |= n >> 2;
+ n |= n >> 4;
+ n |= n >> 8;
+ n |= n >> 16;
+ n |= n >> 32;
+ ++n;
+
+ return n;
+}
+
+template <size_t ELEMENT_SIZE, size_t initial_bytes, typename TAllocator, size_t pad_right_, size_t pad_left_>
+class PODArrayBase : /*private boost::noncopyable,*/ private TAllocator /// empty base optimization
+{
+protected:
+ /// Round padding up to an whole number of elements to simplify arithmetic.
+ static constexpr size_t pad_right = integerRoundUp(pad_right_, ELEMENT_SIZE);
+ /// pad_left is also rounded up to 16 bytes to maintain alignment of allocated memory.
+ static constexpr size_t pad_left = integerRoundUp(integerRoundUp(pad_left_, ELEMENT_SIZE), 16);
+ /// Empty array will point to this static memory as padding and begin/end.
+ static constexpr char * null = const_cast<char *>(empty_pod_array) + pad_left;
+
+ static_assert(pad_left <= empty_pod_array_size && "Left Padding exceeds empty_pod_array_size. Is the element size too large?");
+
+ // If we are using allocator with inline memory, the minimal size of
+ // array must be in sync with the size of this memory.
+ static_assert(allocatorInitialBytes<TAllocator> == 0
+ || allocatorInitialBytes<TAllocator> == initial_bytes);
+
+ char * c_start = null; /// Does not include pad_left.
+ char * c_end = null;
+ char * c_end_of_storage = null; /// Does not include pad_right.
+
+ /// The amount of memory occupied by the num_elements of the elements.
+ static size_t byte_size(size_t num_elements)
+ {
+ size_t amount;
+ if (__builtin_mul_overflow(num_elements, ELEMENT_SIZE, &amount))
+ throw Exception("Amount of memory requested to allocate is more than allowed");
+ return amount;
+ }
+
+ /// Minimum amount of memory to allocate for num_elements, including padding.
+ static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right + pad_left; }
+
+ void alloc_for_num_elements(size_t num_elements)
+ {
+ alloc(minimum_memory_for_elements(num_elements));
+ }
+
+ template <typename ... TAllocatorParams>
+ void alloc(size_t bytes, TAllocatorParams &&... allocator_params)
+ {
+ char * allocated = reinterpret_cast<char *>(TAllocator::alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...));
+
+ c_start = allocated + pad_left;
+ c_end = c_start;
+ c_end_of_storage = allocated + bytes - pad_right;
+
+ if (pad_left)
+ memset(c_start - ELEMENT_SIZE, 0, ELEMENT_SIZE);
+ }
+
+ void dealloc()
+ {
+ if (c_start == null)
+ return;
+
+ unprotect();
+
+ TAllocator::free(c_start - pad_left, allocated_bytes());
+ }
+
+ template <typename ... TAllocatorParams>
+ void realloc(size_t bytes, TAllocatorParams &&... allocator_params)
+ {
+ if (c_start == null)
+ {
+ alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...);
+ return;
+ }
+
+ unprotect();
+
+ ptrdiff_t end_diff = c_end - c_start;
+
+ char * allocated = reinterpret_cast<char *>(
+ TAllocator::realloc(c_start - pad_left, allocated_bytes(), bytes, std::forward<TAllocatorParams>(allocator_params)...));
+
+ c_start = allocated + pad_left;
+ c_end = c_start + end_diff;
+ c_end_of_storage = allocated + bytes - pad_right;
+ }
+
+ bool isInitialized() const
+ {
+ return (c_start != null) && (c_end != null) && (c_end_of_storage != null);
+ }
+
+ bool isAllocatedFromStack() const
+ {
+ static constexpr size_t stack_threshold = TAllocator::getStackThreshold();
+ return (stack_threshold > 0) && (allocated_bytes() <= stack_threshold);
+ }
+
+ template <typename ... TAllocatorParams>
+ void reserveForNextSize(TAllocatorParams &&... allocator_params)
+ {
+ if (empty())
+ {
+ // The allocated memory should be multiplication of ELEMENT_SIZE to hold the element, otherwise,
+ // memory issue such as corruption could appear in edge case.
+ realloc(std::max(integerRoundUp(initial_bytes, ELEMENT_SIZE),
+ minimum_memory_for_elements(1)),
+ std::forward<TAllocatorParams>(allocator_params)...);
+ }
+ else
+ realloc(allocated_bytes() * 2, std::forward<TAllocatorParams>(allocator_params)...);
+ }
+
+#ifndef NDEBUG
+ /// Make memory region readonly with mprotect if it is large enough.
+ /// The operation is slow and performed only for debug builds.
+ void protectImpl(int prot)
+ {
+ static constexpr size_t PROTECT_PAGE_SIZE = 4096;
+
+ char * left_rounded_up = reinterpret_cast<char *>((reinterpret_cast<intptr_t>(c_start) - pad_left + PROTECT_PAGE_SIZE - 1) / PROTECT_PAGE_SIZE * PROTECT_PAGE_SIZE);
+ char * right_rounded_down = reinterpret_cast<char *>((reinterpret_cast<intptr_t>(c_end_of_storage) + pad_right) / PROTECT_PAGE_SIZE * PROTECT_PAGE_SIZE);
+
+ if (right_rounded_down > left_rounded_up)
+ {
+ size_t length = right_rounded_down - left_rounded_up;
+ if (0 != mprotect(left_rounded_up, length, prot))
+ throw Exception("Cannot mprotect memory region");
+ }
+ }
+
+ /// Restore memory protection in destructor or realloc for further reuse by allocator.
+ bool mprotected = false;
+#endif
+
+public:
+ bool empty() const { return c_end == c_start; }
+ size_t size() const { return (c_end - c_start) / ELEMENT_SIZE; }
+ size_t capacity() const { return (c_end_of_storage - c_start) / ELEMENT_SIZE; }
+
+ /// This method is safe to use only for information about memory usage.
+ size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right + pad_left; }
+
+ void clear() { c_end = c_start; }
+
+ template <typename ... TAllocatorParams>
+#if defined(__clang__)
+ ALWAYS_INLINE /// Better performance in clang build, worse performance in gcc build.
+#endif
+ void reserve(size_t n, TAllocatorParams &&... allocator_params)
+ {
+ if (n > capacity())
+ realloc(roundUpToPowerOfTwoOrZero(minimum_memory_for_elements(n)), std::forward<TAllocatorParams>(allocator_params)...);
+ }
+
+ template <typename ... TAllocatorParams>
+ void reserve_exact(size_t n, TAllocatorParams &&... allocator_params)
+ {
+ if (n > capacity())
+ realloc(minimum_memory_for_elements(n), std::forward<TAllocatorParams>(allocator_params)...);
+ }
+
+ template <typename ... TAllocatorParams>
+ void resize(size_t n, TAllocatorParams &&... allocator_params)
+ {
+ reserve(n, std::forward<TAllocatorParams>(allocator_params)...);
+ resize_assume_reserved(n);
+ }
+
+ template <typename ... TAllocatorParams>
+ void resize_exact(size_t n, TAllocatorParams &&... allocator_params)
+ {
+ reserve_exact(n, std::forward<TAllocatorParams>(allocator_params)...);
+ resize_assume_reserved(n);
+ }
+
+ void resize_assume_reserved(const size_t n)
+ {
+ c_end = c_start + byte_size(n);
+ }
+
+ const char * raw_data() const
+ {
+ return c_start;
+ }
+
+ template <typename ... TAllocatorParams>
+ void push_back_raw(const void * ptr, TAllocatorParams &&... allocator_params)
+ {
+ size_t required_capacity = size() + ELEMENT_SIZE;
+ if (unlikely(required_capacity > capacity()))
+ reserve(required_capacity, std::forward<TAllocatorParams>(allocator_params)...);
+
+ memcpy(c_end, ptr, ELEMENT_SIZE);
+ c_end += ELEMENT_SIZE;
+ }
+
+ void protect()
+ {
+#ifndef NDEBUG
+ protectImpl(PROT_READ);
+ mprotected = true;
+#endif
+ }
+
+ void unprotect()
+ {
+#ifndef NDEBUG
+ if (mprotected)
+ protectImpl(PROT_WRITE);
+ mprotected = false;
+#endif
+ }
+
+ template <typename It1, typename It2>
+ inline void assertNotIntersects(It1 from_begin [[maybe_unused]], It2 from_end [[maybe_unused]])
+ {
+#if !defined(NDEBUG)
+ const char * ptr_begin = reinterpret_cast<const char *>(&*from_begin);
+ const char * ptr_end = reinterpret_cast<const char *>(&*from_end);
+
+ /// Also it's safe if the range is empty.
+ assert(!((ptr_begin >= c_start && ptr_begin < c_end) || (ptr_end > c_start && ptr_end <= c_end)) || (ptr_begin == ptr_end));
+#endif
+ }
+
+ ~PODArrayBase()
+ {
+ dealloc();
+ }
+};
+
+template <typename T, size_t initial_bytes, typename TAllocator, size_t pad_right_, size_t pad_left_>
+class PODArray : public PODArrayBase<sizeof(T), initial_bytes, TAllocator, pad_right_, pad_left_>
+{
+protected:
+ using Base = PODArrayBase<sizeof(T), initial_bytes, TAllocator, pad_right_, pad_left_>;
+
+ T * t_start() { return reinterpret_cast<T *>(this->c_start); }
+ T * t_end() { return reinterpret_cast<T *>(this->c_end); }
+
+ const T * t_start() const { return reinterpret_cast<const T *>(this->c_start); }
+ const T * t_end() const { return reinterpret_cast<const T *>(this->c_end); }
+
+public:
+ using value_type = T;
+
+ /// We cannot use boost::iterator_adaptor, because it defeats loop vectorization,
+ /// see https://github.com/ClickHouse/ClickHouse/pull/9442
+
+ using iterator = T *;
+ using const_iterator = const T *;
+
+
+ PODArray() = default;
+
+ PODArray(size_t n)
+ {
+ this->alloc_for_num_elements(n);
+ this->c_end += this->byte_size(n);
+ }
+
+ PODArray(size_t n, const T & x)
+ {
+ this->alloc_for_num_elements(n);
+ assign(n, x);
+ }
+
+ PODArray(const_iterator from_begin, const_iterator from_end)
+ {
+ this->alloc_for_num_elements(from_end - from_begin);
+ insert(from_begin, from_end);
+ }
+
+ PODArray(std::initializer_list<T> il)
+ {
+ this->reserve(std::size(il));
+
+ for (const auto & x : il)
+ {
+ this->push_back(x);
+ }
+ }
+
+ PODArray(PODArray && other)
+ {
+ this->swap(other);
+ }
+
+ PODArray & operator=(PODArray && other)
+ {
+ this->swap(other);
+ return *this;
+ }
+
+ T * data() { return t_start(); }
+ const T * data() const { return t_start(); }
+
+ /// The index is signed to access -1th element without pointer overflow.
+ T & operator[] (ssize_t n)
+ {
+ /// <= size, because taking address of one element past memory range is Ok in C++ (expression like &arr[arr.size()] is perfectly valid).
+ assert((n >= (static_cast<ssize_t>(pad_left_) ? -1 : 0)) && (n <= static_cast<ssize_t>(this->size())));
+ return t_start()[n];
+ }
+
+ const T & operator[] (ssize_t n) const
+ {
+ assert((n >= (static_cast<ssize_t>(pad_left_) ? -1 : 0)) && (n <= static_cast<ssize_t>(this->size())));
+ return t_start()[n];
+ }
+
+ T & front() { return t_start()[0]; }
+ T & back() { return t_end()[-1]; }
+ const T & front() const { return t_start()[0]; }
+ const T & back() const { return t_end()[-1]; }
+
+ iterator begin() { return t_start(); }
+ iterator end() { return t_end(); }
+ const_iterator begin() const { return t_start(); }
+ const_iterator end() const { return t_end(); }
+ const_iterator cbegin() const { return t_start(); }
+ const_iterator cend() const { return t_end(); }
+
+ /// Same as resize, but zeroes new elements.
+ void resize_fill(size_t n)
+ {
+ size_t old_size = this->size();
+ if (n > old_size)
+ {
+ this->reserve(n);
+ memset(this->c_end, 0, this->byte_size(n - old_size));
+ }
+ this->c_end = this->c_start + this->byte_size(n);
+ }
+
+ void resize_fill(size_t n, const T & value)
+ {
+ size_t old_size = this->size();
+ if (n > old_size)
+ {
+ this->reserve(n);
+ std::fill(t_end(), t_end() + n - old_size, value);
+ }
+ this->c_end = this->c_start + this->byte_size(n);
+ }
+
+ template <typename U, typename ... TAllocatorParams>
+ void push_back(U && x, TAllocatorParams &&... allocator_params)
+ {
+ if (unlikely(this->c_end + sizeof(T) > this->c_end_of_storage))
+ this->reserveForNextSize(std::forward<TAllocatorParams>(allocator_params)...);
+
+ new (t_end()) T(std::forward<U>(x));
+ this->c_end += this->byte_size(1);
+ }
+
+ /** This method doesn't allow to pass parameters for Allocator,
+ * and it couldn't be used if Allocator requires custom parameters.
+ */
+ template <typename... Args>
+ void emplace_back(Args &&... args)
+ {
+ if (unlikely(this->c_end + sizeof(T) > this->c_end_of_storage))
+ this->reserveForNextSize();
+
+ new (t_end()) T(std::forward<Args>(args)...);
+ this->c_end += this->byte_size(1);
+ }
+
+ void pop_back()
+ {
+ this->c_end -= this->byte_size(1);
+ }
+
+ /// Do not insert into the array a piece of itself. Because with the resize, the iterators on themselves can be invalidated.
+ template <typename It1, typename It2, typename ... TAllocatorParams>
+ void insertPrepare(It1 from_begin, It2 from_end, TAllocatorParams &&... allocator_params)
+ {
+ this->assertNotIntersects(from_begin, from_end);
+ size_t required_capacity = this->size() + (from_end - from_begin);
+ if (required_capacity > this->capacity())
+ this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward<TAllocatorParams>(allocator_params)...);
+ }
+
+ /// Do not insert into the array a piece of itself. Because with the resize, the iterators on themselves can be invalidated.
+ template <typename It1, typename It2, typename ... TAllocatorParams>
+ void insert(It1 from_begin, It2 from_end, TAllocatorParams &&... allocator_params)
+ {
+ insertPrepare(from_begin, from_end, std::forward<TAllocatorParams>(allocator_params)...);
+ insert_assume_reserved(from_begin, from_end);
+ }
+
+ /// In contrast to 'insert' this method is Ok even for inserting from itself.
+ /// Because we obtain iterators after reserving memory.
+ template <typename Container, typename ... TAllocatorParams>
+ void insertByOffsets(Container && rhs, size_t from_begin, size_t from_end, TAllocatorParams &&... allocator_params)
+ {
+ static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(rhs.front())>>);
+
+ assert(from_end >= from_begin);
+ assert(from_end <= rhs.size());
+
+ size_t required_capacity = this->size() + (from_end - from_begin);
+ if (required_capacity > this->capacity())
+ this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward<TAllocatorParams>(allocator_params)...);
+
+ size_t bytes_to_copy = this->byte_size(from_end - from_begin);
+ if (bytes_to_copy)
+ {
+ memcpy(this->c_end, reinterpret_cast<const void *>(rhs.begin() + from_begin), bytes_to_copy);
+ this->c_end += bytes_to_copy;
+ }
+ }
+
+ /// Works under assumption, that it's possible to read up to 15 excessive bytes after `from_end` and this PODArray is padded.
+ template <typename It1, typename It2, typename ... TAllocatorParams>
+ void insertSmallAllowReadWriteOverflow15(It1 from_begin, It2 from_end, TAllocatorParams &&... allocator_params)
+ {
+ static_assert(pad_right_ >= 15);
+ static_assert(sizeof(T) == sizeof(*from_begin));
+ insertPrepare(from_begin, from_end, std::forward<TAllocatorParams>(allocator_params)...);
+ size_t bytes_to_copy = this->byte_size(from_end - from_begin);
+ memcpySmallAllowReadWriteOverflow15(this->c_end, reinterpret_cast<const void *>(&*from_begin), bytes_to_copy);
+ this->c_end += bytes_to_copy;
+ }
+
+ /// Do not insert into the array a piece of itself. Because with the resize, the iterators on themselves can be invalidated.
+ template <typename It1, typename It2>
+ void insert(iterator it, It1 from_begin, It2 from_end)
+ {
+ static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(*from_begin)>>);
+
+ size_t bytes_to_copy = this->byte_size(from_end - from_begin);
+ if (!bytes_to_copy)
+ return;
+
+ size_t bytes_to_move = this->byte_size(end() - it);
+
+ insertPrepare(from_begin, from_end);
+
+ if (unlikely(bytes_to_move))
+ memmove(this->c_end + bytes_to_copy - bytes_to_move, this->c_end - bytes_to_move, bytes_to_move);
+
+ memcpy(this->c_end - bytes_to_move, reinterpret_cast<const void *>(&*from_begin), bytes_to_copy);
+
+ this->c_end += bytes_to_copy;
+ }
+
+ template <typename ... TAllocatorParams>
+ void insertFromItself(iterator from_begin, iterator from_end, TAllocatorParams && ... allocator_params)
+ {
+ static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(*from_begin)>>);
+
+ /// Convert iterators to indexes because reserve can invalidate iterators
+ size_t start_index = from_begin - begin();
+ size_t end_index = from_end - begin();
+ size_t copy_size = end_index - start_index;
+
+ assert(start_index <= end_index);
+
+ size_t required_capacity = this->size() + copy_size;
+ if (required_capacity > this->capacity())
+ this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward<TAllocatorParams>(allocator_params)...);
+
+ size_t bytes_to_copy = this->byte_size(copy_size);
+ if (bytes_to_copy)
+ {
+ auto begin = this->c_start + this->byte_size(start_index);
+ memcpy(this->c_end, reinterpret_cast<const void *>(&*begin), bytes_to_copy);
+ this->c_end += bytes_to_copy;
+ }
+ }
+
+ template <typename It1, typename It2>
+ void insert_assume_reserved(It1 from_begin, It2 from_end)
+ {
+ static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(*from_begin)>>);
+ this->assertNotIntersects(from_begin, from_end);
+
+ size_t bytes_to_copy = this->byte_size(from_end - from_begin);
+ if (bytes_to_copy)
+ {
+ memcpy(this->c_end, reinterpret_cast<const void *>(&*from_begin), bytes_to_copy);
+ this->c_end += bytes_to_copy;
+ }
+ }
+
+ template <typename... TAllocatorParams>
+ void swap(PODArray & rhs, TAllocatorParams &&... allocator_params)
+ {
+#ifndef NDEBUG
+ this->unprotect();
+ rhs.unprotect();
+#endif
+
+ /// Swap two PODArray objects, arr1 and arr2, that satisfy the following conditions:
+ /// - The elements of arr1 are stored on stack.
+ /// - The elements of arr2 are stored on heap.
+ auto swap_stack_heap = [&](PODArray & arr1, PODArray & arr2)
+ {
+ size_t stack_size = arr1.size();
+ size_t stack_allocated = arr1.allocated_bytes();
+
+ size_t heap_size = arr2.size();
+ size_t heap_allocated = arr2.allocated_bytes();
+
+ /// Keep track of the stack content we have to copy.
+ char * stack_c_start = arr1.c_start;
+
+ /// arr1 takes ownership of the heap memory of arr2.
+ arr1.c_start = arr2.c_start;
+ arr1.c_end_of_storage = arr1.c_start + heap_allocated - arr2.pad_right - arr2.pad_left;
+ arr1.c_end = arr1.c_start + this->byte_size(heap_size);
+
+ /// Allocate stack space for arr2.
+ arr2.alloc(stack_allocated, std::forward<TAllocatorParams>(allocator_params)...);
+ /// Copy the stack content.
+ memcpy(arr2.c_start, stack_c_start, this->byte_size(stack_size));
+ arr2.c_end = arr2.c_start + this->byte_size(stack_size);
+ };
+
+ auto do_move = [&](PODArray & src, PODArray & dest)
+ {
+ if (src.isAllocatedFromStack())
+ {
+ dest.dealloc();
+ dest.alloc(src.allocated_bytes(), std::forward<TAllocatorParams>(allocator_params)...);
+ memcpy(dest.c_start, src.c_start, this->byte_size(src.size()));
+ dest.c_end = dest.c_start + this->byte_size(src.size());
+
+ src.c_start = Base::null;
+ src.c_end = Base::null;
+ src.c_end_of_storage = Base::null;
+ }
+ else
+ {
+ std::swap(dest.c_start, src.c_start);
+ std::swap(dest.c_end, src.c_end);
+ std::swap(dest.c_end_of_storage, src.c_end_of_storage);
+ }
+ };
+
+ if (!this->isInitialized() && !rhs.isInitialized())
+ {
+ return;
+ }
+ else if (!this->isInitialized() && rhs.isInitialized())
+ {
+ do_move(rhs, *this);
+ return;
+ }
+ else if (this->isInitialized() && !rhs.isInitialized())
+ {
+ do_move(*this, rhs);
+ return;
+ }
+
+ if (this->isAllocatedFromStack() && rhs.isAllocatedFromStack())
+ {
+ size_t min_size = std::min(this->size(), rhs.size());
+ size_t max_size = std::max(this->size(), rhs.size());
+
+ for (size_t i = 0; i < min_size; ++i)
+ std::swap(this->operator[](i), rhs[i]);
+
+ if (this->size() == max_size)
+ {
+ for (size_t i = min_size; i < max_size; ++i)
+ rhs[i] = this->operator[](i);
+ }
+ else
+ {
+ for (size_t i = min_size; i < max_size; ++i)
+ this->operator[](i) = rhs[i];
+ }
+
+ size_t lhs_size = this->size();
+ size_t lhs_allocated = this->allocated_bytes();
+
+ size_t rhs_size = rhs.size();
+ size_t rhs_allocated = rhs.allocated_bytes();
+
+ this->c_end_of_storage = this->c_start + rhs_allocated - Base::pad_right - Base::pad_left;
+ rhs.c_end_of_storage = rhs.c_start + lhs_allocated - Base::pad_right - Base::pad_left;
+
+ this->c_end = this->c_start + this->byte_size(rhs_size);
+ rhs.c_end = rhs.c_start + this->byte_size(lhs_size);
+ }
+ else if (this->isAllocatedFromStack() && !rhs.isAllocatedFromStack())
+ {
+ swap_stack_heap(*this, rhs);
+ }
+ else if (!this->isAllocatedFromStack() && rhs.isAllocatedFromStack())
+ {
+ swap_stack_heap(rhs, *this);
+ }
+ else
+ {
+ std::swap(this->c_start, rhs.c_start);
+ std::swap(this->c_end, rhs.c_end);
+ std::swap(this->c_end_of_storage, rhs.c_end_of_storage);
+ }
+ }
+
+ template <typename... TAllocatorParams>
+ void assign(size_t n, const T & x, TAllocatorParams &&... allocator_params)
+ {
+ this->resize_exact(n, std::forward<TAllocatorParams>(allocator_params)...);
+ std::fill(begin(), end(), x);
+ }
+
+ template <typename It1, typename It2, typename... TAllocatorParams>
+ void assign(It1 from_begin, It2 from_end, TAllocatorParams &&... allocator_params)
+ {
+ static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(*from_begin)>>);
+ this->assertNotIntersects(from_begin, from_end);
+
+ size_t required_capacity = from_end - from_begin;
+ if (required_capacity > this->capacity())
+ this->reserve_exact(required_capacity, std::forward<TAllocatorParams>(allocator_params)...);
+
+ size_t bytes_to_copy = this->byte_size(required_capacity);
+ if (bytes_to_copy)
+ {
+ memcpy(this->c_start, reinterpret_cast<const void *>(&*from_begin), bytes_to_copy);
+ this->c_end = this->c_start + bytes_to_copy;
+ }
+ }
+
+ // ISO C++ has strict ambiguity rules, thus we cannot apply TAllocatorParams here.
+ void assign(const PODArray & from)
+ {
+ assign(from.begin(), from.end());
+ }
+
+ void erase(const_iterator first, const_iterator last)
+ {
+ iterator first_no_const = const_cast<iterator>(first);
+ iterator last_no_const = const_cast<iterator>(last);
+
+ size_t items_to_move = end() - last;
+
+ while (items_to_move != 0)
+ {
+ *first_no_const = *last_no_const;
+
+ ++first_no_const;
+ ++last_no_const;
+
+ --items_to_move;
+ }
+
+ this->c_end = reinterpret_cast<char *>(first_no_const);
+ }
+
+ void erase(const_iterator pos)
+ {
+ this->erase(pos, pos + 1);
+ }
+
+ bool operator== (const PODArray & rhs) const
+ {
+ if (this->size() != rhs.size())
+ return false;
+
+ const_iterator lhs_it = begin();
+ const_iterator rhs_it = rhs.begin();
+
+ while (lhs_it != end())
+ {
+ if (*lhs_it != *rhs_it)
+ return false;
+
+ ++lhs_it;
+ ++rhs_it;
+ }
+
+ return true;
+ }
+
+ bool operator!= (const PODArray & rhs) const
+ {
+ return !operator==(rhs);
+ }
+};
+
+template <typename T, size_t initial_bytes, typename TAllocator, size_t pad_right_, size_t pad_left_>
+void swap(PODArray<T, initial_bytes, TAllocator, pad_right_, pad_left_> & lhs, PODArray<T, initial_bytes, TAllocator, pad_right_, pad_left_> & rhs)
+{
+ lhs.swap(rhs);
+}
+#pragma GCC diagnostic pop
+
+/// Prevent implicit template instantiation of PODArray for common numeric types
+
+extern template class PODArray<UInt8, 4096, Allocator<false>, 15, 16>;
+extern template class PODArray<UInt16, 4096, Allocator<false>, 15, 16>;
+extern template class PODArray<UInt32, 4096, Allocator<false>, 15, 16>;
+extern template class PODArray<UInt64, 4096, Allocator<false>, 15, 16>;
+
+extern template class PODArray<Int8, 4096, Allocator<false>, 15, 16>;
+extern template class PODArray<Int16, 4096, Allocator<false>, 15, 16>;
+extern template class PODArray<Int32, 4096, Allocator<false>, 15, 16>;
+extern template class PODArray<Int64, 4096, Allocator<false>, 15, 16>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/PODArray_fwd.h b/ydb/library/arrow_clickhouse/Common/PODArray_fwd.h
new file mode 100644
index 00000000000..8be2d0590a0
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/PODArray_fwd.h
@@ -0,0 +1,40 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+/**
+ * This file contains some using-declarations that define various kinds of
+ * PODArray.
+ */
+
+#include <common/types.h>
+#include <Common/Allocator_fwd.h>
+
+namespace CH
+{
+
+inline constexpr size_t integerRoundUp(size_t value, size_t dividend)
+{
+ return ((value + dividend - 1) / dividend) * dividend;
+}
+
+template <typename T, size_t initial_bytes = 4096,
+ typename TAllocator = Allocator<false>, size_t pad_right_ = 0,
+ size_t pad_left_ = 0>
+class PODArray;
+
+/** For columns. Padding is enough to read and write xmm-register at the address of the last element. */
+template <typename T, size_t initial_bytes = 4096, typename TAllocator = Allocator<false>>
+using PaddedPODArray = PODArray<T, initial_bytes, TAllocator, 15, 16>;
+
+/** A helper for declaring PODArray that uses inline memory.
+ * The initial size is set to use all the inline bytes, since using less would
+ * only add some extra allocation calls.
+ */
+template <typename T, size_t inline_bytes,
+ size_t rounded_bytes = integerRoundUp(inline_bytes, sizeof(T))>
+using PODArrayWithStackMemory = PODArray<T, rounded_bytes,
+ AllocatorWithStackMemory<Allocator<false>, rounded_bytes, alignof(T)>>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/SipHash.h b/ydb/library/arrow_clickhouse/Common/SipHash.h
new file mode 100644
index 00000000000..ac9c21c7d90
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/SipHash.h
@@ -0,0 +1,219 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+/** SipHash is a fast cryptographic hash function for short strings.
+ * Taken from here: https://www.131002.net/siphash/
+ *
+ * This is SipHash 2-4 variant.
+ *
+ * Two changes are made:
+ * - returns also 128 bits, not only 64;
+ * - done streaming (can be calculated in parts).
+ *
+ * On short strings (URL, search phrases) more than 3 times faster than MD5 from OpenSSL.
+ * (~ 700 MB/sec, 15 million strings per second)
+ */
+
+#include <common/types.h>
+#include <common/unaligned.h>
+#include <string>
+#include <type_traits>
+#include <cstddef>
+
+namespace CH
+{
+
+#define ROTL(x, b) static_cast<UInt64>(((x) << (b)) | ((x) >> (64 - (b))))
+
+#define SIPROUND \
+ do \
+ { \
+ v0 += v1; v1 = ROTL(v1, 13); v1 ^= v0; v0 = ROTL(v0, 32); \
+ v2 += v3; v3 = ROTL(v3, 16); v3 ^= v2; \
+ v0 += v3; v3 = ROTL(v3, 21); v3 ^= v0; \
+ v2 += v1; v1 = ROTL(v1, 17); v1 ^= v2; v2 = ROTL(v2, 32); \
+ } while(0)
+
+
+class SipHash
+{
+private:
+ /// State.
+ UInt64 v0;
+ UInt64 v1;
+ UInt64 v2;
+ UInt64 v3;
+
+ /// How many bytes have been processed.
+ UInt64 cnt;
+
+ /// The current 8 bytes of input data.
+ union
+ {
+ UInt64 current_word;
+ UInt8 current_bytes[8];
+ };
+
+ ALWAYS_INLINE void finalize()
+ {
+ /// In the last free byte, we write the remainder of the division by 256.
+ current_bytes[7] = cnt;
+
+ v3 ^= current_word;
+ SIPROUND;
+ SIPROUND;
+ v0 ^= current_word;
+
+ v2 ^= 0xff;
+ SIPROUND;
+ SIPROUND;
+ SIPROUND;
+ SIPROUND;
+ }
+
+public:
+ /// Arguments - seed.
+ SipHash(UInt64 k0 = 0, UInt64 k1 = 0)
+ {
+ /// Initialize the state with some random bytes and seed.
+ v0 = 0x736f6d6570736575ULL ^ k0;
+ v1 = 0x646f72616e646f6dULL ^ k1;
+ v2 = 0x6c7967656e657261ULL ^ k0;
+ v3 = 0x7465646279746573ULL ^ k1;
+
+ cnt = 0;
+ current_word = 0;
+ }
+
+ void update(const char * data, UInt64 size)
+ {
+ const char * end = data + size;
+
+ /// We'll finish to process the remainder of the previous update, if any.
+ if (cnt & 7)
+ {
+ while (cnt & 7 && data < end)
+ {
+ current_bytes[cnt & 7] = *data;
+ ++data;
+ ++cnt;
+ }
+
+ /// If we still do not have enough bytes to an 8-byte word.
+ if (cnt & 7)
+ return;
+
+ v3 ^= current_word;
+ SIPROUND;
+ SIPROUND;
+ v0 ^= current_word;
+ }
+
+ cnt += end - data;
+
+ while (data + 8 <= end)
+ {
+ current_word = unalignedLoad<UInt64>(data);
+
+ v3 ^= current_word;
+ SIPROUND;
+ SIPROUND;
+ v0 ^= current_word;
+
+ data += 8;
+ }
+
+ /// Pad the remainder, which is missing up to an 8-byte word.
+ current_word = 0;
+ switch (end - data)
+ {
+ case 7: current_bytes[6] = data[6]; [[fallthrough]];
+ case 6: current_bytes[5] = data[5]; [[fallthrough]];
+ case 5: current_bytes[4] = data[4]; [[fallthrough]];
+ case 4: current_bytes[3] = data[3]; [[fallthrough]];
+ case 3: current_bytes[2] = data[2]; [[fallthrough]];
+ case 2: current_bytes[1] = data[1]; [[fallthrough]];
+ case 1: current_bytes[0] = data[0]; [[fallthrough]];
+ case 0: break;
+ }
+ }
+
+ template <typename T>
+ void update(const T & x)
+ {
+ update(reinterpret_cast<const char *>(&x), sizeof(x));
+ }
+
+ void update(const std::string & x)
+ {
+ update(x.data(), x.length());
+ }
+
+ /// Get the result in some form. This can only be done once!
+
+ void get128(char * out)
+ {
+ finalize();
+ unalignedStore<UInt64>(out, v0 ^ v1);
+ unalignedStore<UInt64>(out + 8, v2 ^ v3);
+ }
+
+ template <typename T>
+ ALWAYS_INLINE void get128(T & lo, T & hi)
+ {
+ static_assert(sizeof(T) == 8);
+ finalize();
+ lo = v0 ^ v1;
+ hi = v2 ^ v3;
+ }
+
+ template <typename T>
+ ALWAYS_INLINE void get128(T & dst)
+ {
+ static_assert(sizeof(T) == 16);
+ get128(reinterpret_cast<char *>(&dst));
+ }
+
+ UInt64 get64()
+ {
+ finalize();
+ return v0 ^ v1 ^ v2 ^ v3;
+ }
+};
+
+
+#undef ROTL
+#undef SIPROUND
+
+inline void sipHash128(const char * data, const size_t size, char * out)
+{
+ SipHash hash;
+ hash.update(data, size);
+ hash.get128(out);
+}
+
+inline UInt64 sipHash64(const char * data, const size_t size)
+{
+ SipHash hash;
+ hash.update(data, size);
+ return hash.get64();
+}
+
+template <typename T>
+UInt64 sipHash64(const T & x)
+{
+ SipHash hash;
+ hash.update(x);
+ return hash.get64();
+}
+
+inline UInt64 sipHash64(const std::string & s)
+{
+ return sipHash64(s.data(), s.size());
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/Common/memcpySmall.h b/ydb/library/arrow_clickhouse/Common/memcpySmall.h
new file mode 100644
index 00000000000..113342a4d7d
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/Common/memcpySmall.h
@@ -0,0 +1,78 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <string.h>
+
+#ifdef __SSE2__
+#include <emmintrin.h>
+
+namespace CH
+{
+
+/** memcpy function could work suboptimal if all the following conditions are met:
+ * 1. Size of memory region is relatively small (approximately, under 50 bytes).
+ * 2. Size of memory region is not known at compile-time.
+ *
+ * In that case, memcpy works suboptimal by following reasons:
+ * 1. Function is not inlined.
+ * 2. Much time/instructions are spend to process "tails" of data.
+ *
+ * There are cases when function could be implemented in more optimal way, with help of some assumptions.
+ * One of that assumptions - ability to read and write some number of bytes after end of passed memory regions.
+ * Under that assumption, it is possible not to implement difficult code to process tails of data and do copy always by big chunks.
+ *
+ * This case is typical, for example, when many small pieces of data are gathered to single contiguous piece of memory in a loop.
+ * - because each next copy will overwrite excessive data after previous copy.
+ *
+ * Assumption that size of memory region is small enough allows us to not unroll the loop.
+ * This is slower, when size of memory is actually big.
+ *
+ * Use with caution.
+ */
+
+namespace detail
+{
+ inline void memcpySmallAllowReadWriteOverflow15Impl(char * __restrict dst, const char * __restrict src, ssize_t n)
+ {
+ while (n > 0)
+ {
+ _mm_storeu_si128(reinterpret_cast<__m128i *>(dst),
+ _mm_loadu_si128(reinterpret_cast<const __m128i *>(src)));
+
+ dst += 16;
+ src += 16;
+ n -= 16;
+ }
+ }
+}
+
+/** Works under assumption, that it's possible to read up to 15 excessive bytes after end of 'src' region
+ * and to write any garbage into up to 15 bytes after end of 'dst' region.
+ */
+inline void memcpySmallAllowReadWriteOverflow15(void * __restrict dst, const void * __restrict src, size_t n)
+{
+ detail::memcpySmallAllowReadWriteOverflow15Impl(reinterpret_cast<char *>(dst), reinterpret_cast<const char *>(src), n);
+}
+
+/** NOTE There was also a function, that assumes, that you could read any bytes inside same memory page of src.
+ * This function was unused, and also it requires special handling for Valgrind and ASan.
+ */
+
+}
+
+#else /// Implementation for other platforms.
+
+namespace CH
+{
+
+inline void memcpySmallAllowReadWriteOverflow15(void * __restrict dst, const void * __restrict src, size_t n)
+{
+ memcpy(dst, src, n);
+}
+
+}
+
+#endif
diff --git a/ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.cpp b/ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.cpp
new file mode 100644
index 00000000000..b23b70eeb8e
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.cpp
@@ -0,0 +1,127 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#include <DataStreams/AggregatingBlockInputStream.h>
+#include <DataStreams/OneBlockInputStream.h>
+
+namespace CH
+{
+
+/** Combines aggregation states together, turns them into blocks, and outputs streams.
+ * If the aggregation states are two-level, then it produces blocks strictly in order of 'bucket_num'.
+ * (This is important for distributed processing.)
+ * In doing so, it can handle different buckets in parallel, using up to `threads` threads.
+ */
+class MergingAndConvertingBlockInputStream : public IBlockInputStream
+{
+public:
+ /** The input is a set of non-empty sets of partially aggregated data,
+ * which are all either single-level, or are two-level.
+ */
+ MergingAndConvertingBlockInputStream(const Aggregator & aggregator_, ManyAggregatedDataVariants & data_, bool final_)
+ : aggregator(aggregator_), data(data_), final(final_), threads(1)
+ {
+ /// At least we need one arena in first data item per thread
+ if (!data.empty() && threads > data[0]->aggregates_pools.size())
+ {
+ Arenas & first_pool = data[0]->aggregates_pools;
+ for (size_t j = first_pool.size(); j < threads; j++)
+ first_pool.emplace_back(std::make_shared<Arena>());
+ }
+ }
+
+ String getName() const override { return "MergingAndConverting"; }
+
+ Header getHeader() const override { return aggregator.getHeader(final); }
+
+protected:
+ Block readImpl() override
+ {
+ if (data.empty())
+ return {};
+
+ if (current_bucket_num >= NUM_BUCKETS)
+ return {};
+
+ AggregatedDataVariantsPtr & first = data[0];
+
+ if (current_bucket_num == -1)
+ {
+ ++current_bucket_num;
+
+ if (first->type == AggregatedDataVariants::Type::without_key || aggregator.params.overflow_row)
+ {
+ aggregator.mergeWithoutKeyDataImpl(data);
+ return aggregator.prepareBlockAndFillWithoutKey(
+ *first, final, first->type != AggregatedDataVariants::Type::without_key);
+ }
+ }
+
+ {
+ if (current_bucket_num > 0)
+ return {};
+
+ if (first->type == AggregatedDataVariants::Type::without_key)
+ return {};
+
+ ++current_bucket_num;
+
+ #define M(NAME) \
+ else if (first->type == AggregatedDataVariants::Type::NAME) \
+ aggregator.mergeSingleLevelDataImpl<decltype(first->NAME)::element_type>(data);
+ if (false) {} // NOLINT
+ APPLY_FOR_AGGREGATED_VARIANTS(M)
+ #undef M
+ else
+ throw Exception("Unknown aggregated data variant.");
+
+ return aggregator.prepareBlockAndFillSingleLevel(*first, final);
+ }
+ }
+
+private:
+ const Aggregator & aggregator;
+ ManyAggregatedDataVariants data;
+ bool final;
+ size_t threads;
+
+ Int32 current_bucket_num = -1;
+ static constexpr Int32 NUM_BUCKETS = 256;
+};
+
+static std::unique_ptr<IBlockInputStream> mergeAndConvertToBlocks(Aggregator & aggregator,
+ ManyAggregatedDataVariants & data_variants,
+ bool final)
+{
+ ManyAggregatedDataVariants non_empty_data = aggregator.prepareVariantsToMerge(data_variants);
+ if (non_empty_data.empty())
+ return std::make_unique<OneBlockInputStream>(blockFromHeader(aggregator.getHeader(final)));
+ return std::make_unique<MergingAndConvertingBlockInputStream>(aggregator, non_empty_data, final);
+}
+
+Header AggregatingBlockInputStream::getHeader() const
+{
+ return aggregator.getHeader(final);
+}
+
+Block AggregatingBlockInputStream::readImpl()
+{
+ if (!executed)
+ {
+ executed = true;
+ AggregatedDataVariantsPtr data_variants = std::make_shared<AggregatedDataVariants>();
+
+ aggregator.execute(children.back(), *data_variants);
+
+ ManyAggregatedDataVariants many_data { data_variants };
+ impl = mergeAndConvertToBlocks(aggregator, many_data, final);
+ }
+
+ if (isCancelledOrThrowIfKilled() || !impl)
+ return {};
+
+ return impl->read();
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.h b/ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.h
new file mode 100644
index 00000000000..e127c0a7b5f
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.h
@@ -0,0 +1,49 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include "Aggregator.h"
+#include <DataStreams/IBlockStream_fwd.h>
+#include <DataStreams/IBlockInputStream.h>
+
+namespace CH
+{
+
+/** Aggregates the stream of blocks using the specified key columns and aggregate functions.
+ * Columns with aggregate functions adds to the end of the block.
+ * If final = false, the aggregate functions are not finalized, that is, they are not replaced by their value, but contain an intermediate state of calculations.
+ * This is necessary so that aggregation can continue (for example, by combining streams of partially aggregated data).
+ */
+class AggregatingBlockInputStream : public IBlockInputStream
+{
+public:
+ /** keys are taken from the GROUP BY part of the query
+ * Aggregate functions are searched everywhere in the expression.
+ * Columns corresponding to keys and arguments of aggregate functions must already be computed.
+ */
+ AggregatingBlockInputStream(const BlockInputStreamPtr & input, const Aggregator::Params & params_, bool final_)
+ : params(params_), aggregator(params), final(final_)
+ {
+ children.push_back(input);
+ }
+
+ String getName() const override { return "Aggregating"; }
+ Header getHeader() const override;
+
+protected:
+ Block readImpl() override;
+
+ Aggregator::Params params;
+ Aggregator aggregator;
+ bool final;
+
+ bool executed = false;
+
+ /** From here we will get the completed blocks after the aggregation. */
+ std::unique_ptr<IBlockInputStream> impl;
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/DataStreams/CMakeLists.txt b/ydb/library/arrow_clickhouse/DataStreams/CMakeLists.txt
new file mode 100644
index 00000000000..a150b18b741
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/CMakeLists.txt
@@ -0,0 +1,24 @@
+
+# This file was gererated by the build system used internally in the Yandex monorepo.
+# Only simple modifications are allowed (adding source-files to targets, adding simple properties
+# like target_include_directories). These modifications will be ported to original
+# ya.make files by maintainers. Any complex modifications which can't be ported back to the
+# original buildsystem will not be accepted.
+
+
+
+add_library(library-arrow_clickhouse-DataStreams)
+target_include_directories(library-arrow_clickhouse-DataStreams PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/yql/udfs/common/clickhouse/client/base
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse
+)
+target_link_libraries(library-arrow_clickhouse-DataStreams PUBLIC
+ contrib-libs-cxxsupp
+ yutil
+ libs-apache-arrow
+)
+target_sources(library-arrow_clickhouse-DataStreams PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/DataStreams/AggregatingBlockInputStream.cpp
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.cpp
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.cpp
+)
diff --git a/ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.cpp b/ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.cpp
new file mode 100644
index 00000000000..619255403dc
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.cpp
@@ -0,0 +1,90 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#include <DataStreams/IBlockInputStream.h>
+
+namespace CH
+{
+
+
+/// It's safe to access children without mutex as long as these methods are called before first call to `read()` or `readPrefix()`.
+
+
+Block IBlockInputStream::read()
+{
+ Block res;
+ if (isCancelledOrThrowIfKilled())
+ return res;
+
+ res = readImpl();
+ if (!res)
+ {
+ /** If the stream is over, then we will ask all children to abort the execution.
+ * This makes sense when running a query with LIMIT
+ * - there is a situation when all the necessary data has already been read,
+ * but children sources are still working,
+ * herewith they can work in separate threads or even remotely.
+ */
+ cancel(false);
+ }
+
+ return res;
+}
+
+
+void IBlockInputStream::readPrefix()
+{
+ readPrefixImpl();
+
+ forEachChild([&] (IBlockInputStream & child)
+ {
+ child.readPrefix();
+ return false;
+ });
+}
+
+
+void IBlockInputStream::readSuffix()
+{
+ forEachChild([&] (IBlockInputStream & child)
+ {
+ child.readSuffix();
+ return false;
+ });
+
+ readSuffixImpl();
+}
+
+
+void IBlockInputStream::cancel(bool kill)
+{
+#if 0
+ if (kill)
+ is_killed = true;
+#endif
+ bool old_val = false;
+ if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
+ return;
+
+ forEachChild([&] (IBlockInputStream & child)
+ {
+ child.cancel(kill);
+ return false;
+ });
+}
+
+
+bool IBlockInputStream::isCancelled() const
+{
+ return is_cancelled;
+}
+
+bool IBlockInputStream::isCancelledOrThrowIfKilled() const
+{
+ if (!is_cancelled)
+ return false;
+ return true;
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.h b/ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.h
new file mode 100644
index 00000000000..7c11acb094e
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/IBlockInputStream.h
@@ -0,0 +1,118 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+
+#include <DataStreams/IBlockStream_fwd.h>
+
+#include <atomic>
+#include <shared_mutex>
+
+
+namespace CH
+{
+
+/** The stream interface for reading data by blocks from the database.
+ * Relational operations are supposed to be done also as implementations of this interface.
+ * Watches out at how the source of the blocks works.
+ * Lets you get information for profiling: rows per second, blocks per second, megabytes per second, etc.
+ * Allows you to stop reading data (in nested sources).
+ */
+class IBlockInputStream
+{
+public:
+ IBlockInputStream() {}
+ virtual ~IBlockInputStream() {}
+
+ IBlockInputStream(const IBlockInputStream &) = delete;
+ IBlockInputStream & operator=(const IBlockInputStream &) = delete;
+
+ /// To output the data stream transformation tree (query execution plan).
+ virtual String getName() const = 0;
+
+ /** Get data structure of the stream in a form of "header" block (it is also called "sample block").
+ * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values.
+ * It is guaranteed that method "read" returns blocks of exactly that structure.
+ */
+ virtual Header getHeader() const = 0;
+
+ /** Read next block.
+ * If there are no more blocks, return an empty block (for which operator `bool` returns false).
+ * NOTE: Only one thread can read from one instance of IBlockInputStream simultaneously.
+ * This also applies for readPrefix, readSuffix.
+ */
+ Block read();
+
+ /** Read something before starting all data or after the end of all data.
+ * In the `readSuffix` function, you can implement a finalization that can lead to an exception.
+ * readPrefix() must be called before the first call to read().
+ * readSuffix() should be called after read() returns an empty block, or after a call to cancel(), but not during read() execution.
+ */
+
+ /** The default implementation calls readPrefixImpl() on itself, and then readPrefix() recursively for all children.
+ * There are cases when you do not want `readPrefix` of children to be called synchronously, in this function,
+ * but you want them to be called, for example, in separate threads (for parallel initialization of children).
+ * Then overload `readPrefix` function.
+ */
+ virtual void readPrefix();
+
+ /** The default implementation calls recursively readSuffix() on all children, and then readSuffixImpl() on itself.
+ * If this stream calls read() in children in a separate thread, this behavior is usually incorrect:
+ * readSuffix() of the child can not be called at the moment when the same child's read() is executed in another thread.
+ * In this case, you need to override this method so that readSuffix() in children is called, for example, after connecting streams.
+ */
+ virtual void readSuffix();
+
+ /** Ask to abort the receipt of data as soon as possible.
+ * By default - just sets the flag is_cancelled and asks that all children be interrupted.
+ * This function can be called several times, including simultaneously from different threads.
+ * Have two modes:
+ * with kill = false only is_cancelled is set - streams will stop silently with returning some processed data.
+ * with kill = true also is_killed set - queries will stop with exception.
+ */
+ virtual void cancel(bool kill = false);
+
+ bool isCancelled() const;
+ bool isCancelledOrThrowIfKilled() const;
+
+protected:
+ BlockInputStreams children;
+ std::shared_mutex children_mutex;
+
+ std::atomic<bool> is_cancelled{false};
+
+ void addChild(const BlockInputStreamPtr & child)
+ {
+ std::unique_lock lock(children_mutex);
+ children.push_back(child);
+ }
+
+private:
+ /// Derived classes must implement this function.
+ virtual Block readImpl() = 0;
+
+ /// Here you can do a preliminary initialization.
+ virtual void readPrefixImpl() {}
+
+ /// Here you need to do a finalization, which can lead to an exception.
+ virtual void readSuffixImpl() {}
+
+ template <typename F>
+ void forEachChild(F && f)
+ {
+ /// NOTE: Acquire a read lock, therefore f() should be thread safe
+ std::shared_lock lock(children_mutex);
+
+ // Reduce lock scope and avoid recursive locking since that is undefined for shared_mutex.
+ const auto children_copy = children;
+ lock.unlock();
+
+ for (auto & child : children_copy)
+ if (f(*child))
+ return;
+ }
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/DataStreams/IBlockStream_fwd.h b/ydb/library/arrow_clickhouse/DataStreams/IBlockStream_fwd.h
new file mode 100644
index 00000000000..1bf79768c04
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/IBlockStream_fwd.h
@@ -0,0 +1,21 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+
+#include <memory>
+#include <vector>
+
+namespace CH
+{
+
+class IBlockInputStream;
+class IBlockOutputStream;
+
+using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
+using BlockInputStreams = std::vector<BlockInputStreamPtr>;
+using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
+using BlockOutputStreams = std::vector<BlockOutputStreamPtr>;
+
+}
diff --git a/ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.cpp b/ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.cpp
new file mode 100644
index 00000000000..2bd14e1371a
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.cpp
@@ -0,0 +1,43 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#include <DataStreams/MergingAggregatedBlockInputStream.h>
+
+
+namespace CH
+{
+
+Header MergingAggregatedBlockInputStream::getHeader() const
+{
+ return aggregator.getHeader(final);
+}
+
+
+Block MergingAggregatedBlockInputStream::readImpl()
+{
+ if (!executed)
+ {
+ executed = true;
+ AggregatedDataVariants data_variants;
+#if 0
+ Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); };
+ aggregator.setCancellationHook(hook);
+#endif
+ aggregator.mergeStream(children.back(), data_variants);
+ blocks = aggregator.convertToBlocks(data_variants, final);
+ it = blocks.begin();
+ }
+
+ Block res;
+ if (isCancelledOrThrowIfKilled() || it == blocks.end())
+ return res;
+
+ res = std::move(*it);
+ ++it;
+
+ return res;
+}
+
+
+}
diff --git a/ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.h b/ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.h
new file mode 100644
index 00000000000..a4fa0274ac2
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/MergingAggregatedBlockInputStream.h
@@ -0,0 +1,42 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+#include "Aggregator.h"
+#include <DataStreams/IBlockInputStream.h>
+
+
+namespace CH
+{
+
+/** A pre-aggregate stream of blocks in which each block is already aggregated.
+ * Aggregate functions in blocks should not be finalized so that their states can be merged.
+ */
+class MergingAggregatedBlockInputStream : public IBlockInputStream
+{
+public:
+ MergingAggregatedBlockInputStream(const BlockInputStreamPtr & input, const Aggregator::Params & params, bool final_)
+ : aggregator(params), final(final_)
+ {
+ children.push_back(input);
+ }
+
+ String getName() const override { return "MergingAggregated"; }
+
+ Header getHeader() const override;
+
+protected:
+ Block readImpl() override;
+
+private:
+ Aggregator aggregator;
+ bool final;
+
+ bool executed = false;
+ BlocksList blocks;
+ BlocksList::iterator it;
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/DataStreams/OneBlockInputStream.h b/ydb/library/arrow_clickhouse/DataStreams/OneBlockInputStream.h
new file mode 100644
index 00000000000..6735022c46e
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/DataStreams/OneBlockInputStream.h
@@ -0,0 +1,49 @@
+// The code in this file is based on original ClickHouse source code
+// which is licensed under Apache license v2.0
+// See: https://github.com/ClickHouse/ClickHouse/
+
+#pragma once
+#include "arrow_clickhouse_types.h"
+#include <DataStreams/IBlockInputStream.h>
+
+namespace CH
+{
+
+/** A stream of blocks from which you can read one block.
+ * Also see BlocksListBlockInputStream.
+ */
+class OneBlockInputStream : public IBlockInputStream
+{
+public:
+ explicit OneBlockInputStream(Block block_)
+ : block(std::move(block_))
+ {
+ if (!block->Validate().ok())
+ throw Exception("Bad batch in OneBlockInputStream");
+ }
+
+ String getName() const override { return "One"; }
+
+ Header getHeader() const override
+ {
+ if (!block)
+ return {};
+ return block->schema();
+ }
+
+protected:
+ Block readImpl() override
+ {
+ if (has_been_read)
+ return {};
+
+ has_been_read = true;
+ return block;
+ }
+
+private:
+ Block block{};
+ bool has_been_read = false;
+};
+
+}
diff --git a/ydb/library/arrow_clickhouse/README.md b/ydb/library/arrow_clickhouse/README.md
new file mode 100644
index 00000000000..89149bf4f99
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/README.md
@@ -0,0 +1,16 @@
+ClickHouse aggregate functions over Apache Arrow primitives
+--------
+
+This library is a modified ClickHouse (https://github.com/ClickHouse/ClickHouse/) code that uses Apache Arrow
+(https://arrow.apache.org/) primitives instead of ClickHouse native ones. I.e. it uses arrow::RecordBatch
+instead of DB::Block, arrow::Array and arrow::Builder instead of DB::IColumn and so on.
+The redefinition of types is in arrow_clickhouse_types.h header.
+
+The library uses DataStreams primitives that were replaced by processors in ClickHouse 20.3. It's not possible to
+extract processors from ClickHouse code base. It's too monolithic and depends on specific multithreading model.
+
+The core reason of library is posibility to use ClickHouse's GROUP BY code (Aggregator.cpp) and aggregate fucntions
+(AggregateFunctions directory) with minimal modifications over data presented in Apache Arrow formats.
+
+Original ClickHouse support 2-level aggregation and several optiumizations (LowCardinality, Sparse data, LLVM).
+Also it allows to add functions combinators to aggregate functions. Such optimizations are not implemented here yet.
diff --git a/ydb/library/arrow_clickhouse/arrow_clickhouse_types.h b/ydb/library/arrow_clickhouse/arrow_clickhouse_types.h
new file mode 100644
index 00000000000..698503007eb
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/arrow_clickhouse_types.h
@@ -0,0 +1,146 @@
+#pragma once
+#include <cstdint>
+#include <string>
+#include <vector>
+#include <list>
+#include <map>
+#include <stdexcept>
+
+#include <contrib/libs/apache/arrow/cpp/src/arrow/api.h>
+#include <contrib/libs/apache/arrow/cpp/src/arrow/compute/api.h>
+
+#include <common/StringRef.h>
+#include <common/extended_types.h>
+#include <common/defines.h>
+
+#include <Common/PODArray_fwd.h>
+
+namespace CH
+{
+
+using NDB::StringRef;
+using NDB::StringRefHash;
+using NDB::StringRefs;
+
+/// What to do if the limit is exceeded.
+enum class OverflowMode
+{
+ THROW = 0, /// Throw exception.
+ BREAK = 1, /// Abort query execution, return what is.
+
+ /** Only for GROUP BY: do not add new rows to the set,
+ * but continue to aggregate for keys that are already in the set.
+ */
+ ANY = 2,
+};
+
+using Exception = std::runtime_error;
+using ColumnNumbers = std::vector<uint32_t>; // it's vector<size_t> in CH
+using Names = std::vector<std::string>;
+
+using Block = std::shared_ptr<arrow::RecordBatch>;
+using BlocksList = std::list<Block>;
+using Array = arrow::ScalarVector;
+using ColumnWithTypeAndName = arrow::Field;
+using ColumnsWithTypeAndName = arrow::FieldVector;
+using Header = std::shared_ptr<arrow::Schema>;
+using Sizes = std::vector<size_t>;
+
+// TODO: replace with arrow::memory_pool
+class Arena;
+using ArenaPtr = std::shared_ptr<Arena>;
+using ConstArenaPtr = std::shared_ptr<const Arena>;
+using ConstArenas = std::vector<ConstArenaPtr>;
+
+using IColumn = arrow::Array;
+using ColumnPtr = std::shared_ptr<IColumn>;
+using Columns = std::vector<ColumnPtr>;
+using ColumnRawPtrs = std::vector<const IColumn *>;
+
+using MutableColumn = arrow::ArrayBuilder;
+using MutableColumnPtr = std::shared_ptr<arrow::ArrayBuilder>;
+using MutableColumns = std::vector<MutableColumnPtr>;
+
+struct XColumn {
+ using Offset = UInt64;
+ using Offsets = PaddedPODArray<Offset>;
+
+ using ColumnIndex = UInt64;
+ using Selector = PaddedPODArray<ColumnIndex>;
+
+ using Filter = PaddedPODArray<UInt8>;
+};
+
+using ColumnInt8 = arrow::NumericArray<arrow::Int8Type>;
+using ColumnInt16 = arrow::NumericArray<arrow::Int16Type>;
+using ColumnInt32 = arrow::NumericArray<arrow::Int32Type>;
+using ColumnInt64 = arrow::NumericArray<arrow::Int64Type>;
+
+using ColumnUInt8 = arrow::NumericArray<arrow::UInt8Type>;
+using ColumnUInt16 = arrow::NumericArray<arrow::UInt16Type>;
+using ColumnUInt32 = arrow::NumericArray<arrow::UInt32Type>;
+using ColumnUInt64 = arrow::NumericArray<arrow::UInt64Type>;
+
+using ColumnFloat32 = arrow::NumericArray<arrow::FloatType>;
+using ColumnFloat64 = arrow::NumericArray<arrow::DoubleType>;
+
+using ColumnBinary = arrow::BinaryArray;
+using ColumnString = arrow::StringArray;
+using ColumnFixedString = arrow::FixedSizeBinaryArray;
+
+using MutableColumnInt8 = arrow::Int8Builder;
+using MutableColumnInt16 = arrow::Int16Builder;
+using MutableColumnInt32 = arrow::Int32Builder;
+using MutableColumnInt64 = arrow::Int64Builder;
+
+using MutableColumnUInt8 = arrow::UInt8Builder;
+using MutableColumnUInt16 = arrow::UInt16Builder;
+using MutableColumnUInt32 = arrow::UInt32Builder;
+using MutableColumnUInt64 = arrow::UInt64Builder;
+
+using MutableColumnFloat32 = arrow::FloatBuilder;
+using MutableColumnFloat64 = arrow::DoubleBuilder;
+
+using MutableColumnBinary = arrow::BinaryBuilder;
+using MutableColumnString = arrow::StringBuilder;
+using MutableColumnFixedString = arrow::FixedSizeBinaryBuilder;
+
+using IDataType = arrow::DataType;
+using DataTypePtr = std::shared_ptr<IDataType>;
+using DataTypes = arrow::DataTypeVector;
+
+using DataTypeInt8 = arrow::Int8Type;
+using DataTypeInt16 = arrow::Int16Type;
+using DataTypeInt32 = arrow::Int32Type;
+using DataTypeInt64 = arrow::Int64Type;
+
+using DataTypeUInt8 = arrow::UInt8Type;
+using DataTypeUInt16 = arrow::UInt16Type;
+using DataTypeUInt32 = arrow::UInt32Type;
+using DataTypeUInt64 = arrow::UInt64Type;
+
+using DataTypeFixedString = arrow::FixedSizeBinaryType;
+
+inline Columns columnsFromHeader(const Header& schema, size_t num_rows = 0)
+{
+ std::vector<std::shared_ptr<arrow::Array>> columns;
+ columns.reserve(schema->num_fields());
+
+ for (auto& field : schema->fields()) {
+ columns.emplace_back(*arrow::MakeArrayOfNull(field->type(), num_rows));
+ }
+ return columns;
+}
+
+inline Block blockFromHeader(const Header& schema, size_t num_rows = 0)
+{
+ return arrow::RecordBatch::Make(schema, num_rows, columnsFromHeader(schema, num_rows));
+}
+
+template <typename To, typename From>
+inline To assert_cast(From && from)
+{
+ return static_cast<To>(from);
+}
+
+}
diff --git a/ydb/library/arrow_clickhouse/ut/CMakeLists.darwin.txt b/ydb/library/arrow_clickhouse/ut/CMakeLists.darwin.txt
new file mode 100644
index 00000000000..1a9a9c7126e
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/ut/CMakeLists.darwin.txt
@@ -0,0 +1,41 @@
+
+# This file was gererated by the build system used internally in the Yandex monorepo.
+# Only simple modifications are allowed (adding source-files to targets, adding simple properties
+# like target_include_directories). These modifications will be ported to original
+# ya.make files by maintainers. Any complex modifications which can't be ported back to the
+# original buildsystem will not be accepted.
+
+
+
+add_executable(ydb-library-arrow_clickhouse-ut)
+target_include_directories(ydb-library-arrow_clickhouse-ut PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse
+)
+target_link_libraries(ydb-library-arrow_clickhouse-ut PUBLIC
+ contrib-libs-cxxsupp
+ yutil
+ library-cpp-cpuid_check
+ cpp-testing-unittest_main
+ ydb-library-arrow_clickhouse
+)
+target_link_options(ydb-library-arrow_clickhouse-ut PRIVATE
+ -Wl,-no_deduplicate
+ -Wl,-sdk_version,10.15
+ -fPIC
+ -fPIC
+)
+target_sources(ydb-library-arrow_clickhouse-ut PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/ut_aggregator.cpp
+)
+add_test(
+ NAME
+ ydb-library-arrow_clickhouse-ut
+ COMMAND
+ ydb-library-arrow_clickhouse-ut
+ --print-before-suite
+ --print-before-test
+ --fork-tests
+ --print-times
+ --show-fails
+)
+vcs_info(ydb-library-arrow_clickhouse-ut)
diff --git a/ydb/library/arrow_clickhouse/ut/CMakeLists.linux.txt b/ydb/library/arrow_clickhouse/ut/CMakeLists.linux.txt
new file mode 100644
index 00000000000..37e06a08220
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/ut/CMakeLists.linux.txt
@@ -0,0 +1,47 @@
+
+# This file was gererated by the build system used internally in the Yandex monorepo.
+# Only simple modifications are allowed (adding source-files to targets, adding simple properties
+# like target_include_directories). These modifications will be ported to original
+# ya.make files by maintainers. Any complex modifications which can't be ported back to the
+# original buildsystem will not be accepted.
+
+
+
+add_executable(ydb-library-arrow_clickhouse-ut)
+target_include_directories(ydb-library-arrow_clickhouse-ut PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse
+)
+target_link_libraries(ydb-library-arrow_clickhouse-ut PUBLIC
+ contrib-libs-cxxsupp
+ yutil
+ cpp-malloc-tcmalloc
+ libs-tcmalloc-no_percpu_cache
+ library-cpp-cpuid_check
+ cpp-testing-unittest_main
+ ydb-library-arrow_clickhouse
+)
+target_link_options(ydb-library-arrow_clickhouse-ut PRIVATE
+ -ldl
+ -lrt
+ -Wl,--no-as-needed
+ -fPIC
+ -fPIC
+ -lpthread
+ -lrt
+ -ldl
+)
+target_sources(ydb-library-arrow_clickhouse-ut PRIVATE
+ ${CMAKE_SOURCE_DIR}/ydb/library/arrow_clickhouse/ut_aggregator.cpp
+)
+add_test(
+ NAME
+ ydb-library-arrow_clickhouse-ut
+ COMMAND
+ ydb-library-arrow_clickhouse-ut
+ --print-before-suite
+ --print-before-test
+ --fork-tests
+ --print-times
+ --show-fails
+)
+vcs_info(ydb-library-arrow_clickhouse-ut)
diff --git a/ydb/library/arrow_clickhouse/ut/CMakeLists.txt b/ydb/library/arrow_clickhouse/ut/CMakeLists.txt
new file mode 100644
index 00000000000..fc7b1ee73ce
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/ut/CMakeLists.txt
@@ -0,0 +1,13 @@
+
+# This file was gererated by the build system used internally in the Yandex monorepo.
+# Only simple modifications are allowed (adding source-files to targets, adding simple properties
+# like target_include_directories). These modifications will be ported to original
+# ya.make files by maintainers. Any complex modifications which can't be ported back to the
+# original buildsystem will not be accepted.
+
+
+if (APPLE)
+ include(CMakeLists.darwin.txt)
+elseif (UNIX AND NOT APPLE)
+ include(CMakeLists.linux.txt)
+endif()
diff --git a/ydb/library/arrow_clickhouse/ut_aggregator.cpp b/ydb/library/arrow_clickhouse/ut_aggregator.cpp
new file mode 100644
index 00000000000..ca25f3aff3b
--- /dev/null
+++ b/ydb/library/arrow_clickhouse/ut_aggregator.cpp
@@ -0,0 +1,365 @@
+#include <array>
+#include <memory>
+#include <vector>
+#include <iostream>
+#include <iomanip>
+
+#include <library/cpp/testing/unittest/registar.h>
+
+#include "Aggregator.h"
+#include "DataStreams/OneBlockInputStream.h"
+#include "DataStreams/AggregatingBlockInputStream.h"
+#include "DataStreams/MergingAggregatedBlockInputStream.h"
+#include "AggregateFunctions/AggregateFunctionCount.h"
+#include "AggregateFunctions/AggregateFunctionMinMaxAny.h"
+#include "AggregateFunctions/AggregateFunctionSum.h"
+#include "AggregateFunctions/AggregateFunctionAvg.h"
+
+namespace CH {
+
+void RegisterAggregates(arrow::compute::FunctionRegistry * registry = nullptr) {
+ if (!registry)
+ registry = arrow::compute::GetFunctionRegistry();
+
+ registry->AddFunction(std::make_shared<CH::WrappedCount>("ch.count")).ok();
+ registry->AddFunction(std::make_shared<CH::WrappedMin>("ch.min")).ok();
+ registry->AddFunction(std::make_shared<CH::WrappedMax>("ch.max")).ok();
+ registry->AddFunction(std::make_shared<CH::WrappedAny>("ch.any")).ok();
+ registry->AddFunction(std::make_shared<CH::WrappedSum>("ch.sum")).ok();
+ registry->AddFunction(std::make_shared<CH::WrappedAvg>("ch.avg")).ok();
+}
+
+// {i16, ui32, s1, s2}
+Block makeTestBlock(size_t num_rows) {
+ std::vector<std::string> strings = {"abc", "def", "abcd", "defg", "ac"};
+
+ arrow::FieldVector fields;
+ arrow::ArrayVector columns;
+
+ {
+ auto field = std::make_shared<arrow::Field>("i16", arrow::int16());
+ arrow::Int16Builder col;
+ col.Reserve(num_rows).ok();
+
+ for (size_t i = 0; i < num_rows; ++i)
+ col.Append(i % 9).ok();
+
+ fields.emplace_back(std::move(field));
+ columns.emplace_back(std::move(*col.Finish()));
+ }
+
+ {
+ auto field = std::make_shared<arrow::Field>("ui32", arrow::uint32());
+ arrow::UInt32Builder col;
+ col.Reserve(num_rows).ok();
+
+ for (size_t i = 0; i < num_rows; ++i)
+ col.Append(i % 7).ok();
+
+ fields.emplace_back(std::move(field));
+ columns.emplace_back(std::move(*col.Finish()));
+ }
+
+ {
+ auto field = std::make_shared<arrow::Field>("s1", arrow::binary());
+ arrow::BinaryBuilder col;
+ col.Reserve(num_rows).ok();
+
+ for (size_t i = 0; i < num_rows; ++i)
+ col.Append(strings[i % strings.size()]).ok();
+
+ fields.emplace_back(std::move(field));
+ columns.emplace_back(std::move(*col.Finish()));
+ }
+
+ {
+ auto field = std::make_shared<arrow::Field>("s2", arrow::binary());
+ arrow::BinaryBuilder col;
+ col.Reserve(num_rows).ok();
+
+ for (size_t i = 0; i < num_rows; ++i)
+ col.Append(strings[i % 3]).ok();
+
+ fields.emplace_back(std::move(field));
+ columns.emplace_back(std::move(*col.Finish()));
+ }
+
+ return arrow::RecordBatch::Make(std::make_shared<arrow::Schema>(fields), num_rows, columns);
+}
+
+AggregateDescription MakeCountDescription(const std::string & column_name = "cnt")
+{
+ auto * registry = arrow::compute::GetFunctionRegistry();
+ auto func = registry->GetFunction("ch.count");
+ auto wrapped = std::static_pointer_cast<ArrowAggregateFunctionWrapper>(*func);
+
+ DataTypes empty_list_of_types;
+ return AggregateDescription {
+ .function = wrapped->getHouseFunction(empty_list_of_types),
+ .column_name = column_name
+ };
+}
+
+AggregateDescription MakeMinMaxAnyDescription(const std::string & agg_name, DataTypePtr data_type,
+ uint32_t column_id)
+{
+ auto * registry = arrow::compute::GetFunctionRegistry();
+ auto func = registry->GetFunction(agg_name);
+ auto wrapped = std::static_pointer_cast<ArrowAggregateFunctionWrapper>(*func);
+
+ DataTypes list_of_types = {data_type};
+ return AggregateDescription {
+ .function = wrapped->getHouseFunction(list_of_types),
+ .arguments = {column_id},
+ .column_name = "res_" + agg_name
+ };
+}
+
+AggregateDescription MakeSumDescription(DataTypePtr data_type, uint32_t column_id,
+ const std::string & column_name = "res_sum")
+{
+ auto * registry = arrow::compute::GetFunctionRegistry();
+ auto func = registry->GetFunction("ch.sum");
+ auto wrapped = std::static_pointer_cast<ArrowAggregateFunctionWrapper>(*func);
+
+ DataTypes list_of_types = {data_type};
+ return AggregateDescription {
+ .function = wrapped->getHouseFunction(list_of_types),
+ .arguments = {column_id},
+ .column_name = column_name
+ };
+}
+
+AggregateDescription MakeAvgDescription(DataTypePtr data_type, uint32_t column_id,
+ const std::string & column_name = "res_avg")
+{
+ auto * registry = arrow::compute::GetFunctionRegistry();
+ auto func = registry->GetFunction("ch.avg");
+ auto wrapped = std::static_pointer_cast<ArrowAggregateFunctionWrapper>(*func);
+
+ DataTypes list_of_types = {data_type};
+ return AggregateDescription {
+ .function = wrapped->getHouseFunction(list_of_types),
+ .arguments = {column_id},
+ .column_name = column_name
+ };
+}
+
+BlockInputStreamPtr MakeAggregatingStream(const BlockInputStreamPtr & stream,
+ const ColumnNumbers & agg_keys,
+ const AggregateDescriptions & aggregate_descriptions)
+{
+ Header src_header = stream->getHeader();
+ Aggregator::Params agg_params(false, src_header, agg_keys, aggregate_descriptions, false);
+ BlockInputStreamPtr agg_stream = std::make_shared<AggregatingBlockInputStream>(stream, agg_params, false);
+
+ ColumnNumbers merge_keys;
+ {
+ Header agg_header = agg_stream->getHeader();
+ for (const auto & key : agg_keys)
+ merge_keys.push_back(agg_header->GetFieldIndex(src_header->field(key)->name()));
+ }
+
+ Aggregator::Params merge_params(true, agg_stream->getHeader(), merge_keys, aggregate_descriptions, false);
+ return std::make_shared<MergingAggregatedBlockInputStream>(agg_stream, merge_params, true);
+}
+
+bool TestExecute(const Block & block, const ColumnNumbers & agg_keys)
+{
+ try
+ {
+ BlockInputStreamPtr stream = std::make_shared<OneBlockInputStream>(block);
+
+ AggregateDescription aggregate_description = MakeCountDescription();
+ Aggregator::Params params(false, stream->getHeader(), agg_keys, {aggregate_description}, false);
+ Aggregator aggregator(params);
+
+ AggregatedDataVariants aggregated_data_variants;
+
+ {
+ //Stopwatch stopwatch;
+ //stopwatch.start();
+
+ aggregator.execute(stream, aggregated_data_variants);
+
+ //stopwatch.stop();
+ //std::cout << std::fixed << std::setprecision(2)
+ // << "Elapsed " << stopwatch.elapsedSeconds() << " sec."
+ // << ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
+ // << std::endl;
+ }
+ }
+ catch (const Exception & e)
+ {
+ std::cerr << e.what() << std::endl;
+ return false;
+ }
+
+ return true;
+}
+
+size_t TestAggregate(const Block & block, const ColumnNumbers & agg_keys, const AggregateDescription & description)
+{
+ size_t rows = 0;
+
+ try
+ {
+ std::cerr << "aggregate by keys: ";
+ for (auto& key : agg_keys) {
+ std::cerr << key << " ";
+ }
+ std::cerr << std::endl;
+
+ auto stream = MakeAggregatingStream(std::make_shared<OneBlockInputStream>(block), agg_keys, {description});
+
+ while (auto block = stream->read()) {
+ std::cerr << "result rows: " << block->num_rows() << std::endl;
+ rows += block->num_rows();
+ }
+ }
+ catch (const Exception & e)
+ {
+ std::cerr << e.what() << std::endl;
+ return 0;
+ }
+
+ return rows;
+}
+
+}
+
+
+Y_UNIT_TEST_SUITE(CH_Aggregator) {
+ Y_UNIT_TEST(ExecuteCount) {
+ CH::RegisterAggregates();
+
+ auto block = CH::makeTestBlock(1000);
+
+ UNIT_ASSERT(CH::TestExecute(block, {0, 1}));
+ UNIT_ASSERT(CH::TestExecute(block, {1, 0}));
+ UNIT_ASSERT(CH::TestExecute(block, {0, 2}));
+ UNIT_ASSERT(CH::TestExecute(block, {2, 0}));
+ UNIT_ASSERT(CH::TestExecute(block, {2, 3}));
+ UNIT_ASSERT(CH::TestExecute(block, {0, 1, 2, 3}));
+ }
+
+ Y_UNIT_TEST(AggregateCount) {
+ CH::RegisterAggregates();
+
+ auto block = CH::makeTestBlock(1000);
+
+ auto agg_count = CH::MakeCountDescription();
+
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1}, agg_count), 9*7);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {1, 0}, agg_count), 7*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 2}, agg_count), 9*5);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 0}, agg_count), 5*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 3}, agg_count), 5*3);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1, 2, 3}, agg_count), 9*7*5);
+ }
+
+ Y_UNIT_TEST(AggregateMin) {
+ CH::RegisterAggregates();
+
+ auto block = CH::makeTestBlock(1000);
+
+ for (int i = 0; i < block->num_columns(); ++i) {
+ auto type = block->column(i)->type();
+ auto agg_descr = CH::MakeMinMaxAnyDescription("ch.min", type, i);
+
+ UNIT_ASSERT(agg_descr.function);
+ UNIT_ASSERT_VALUES_EQUAL(agg_descr.arguments.size(), 1);
+
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1}, agg_descr), 9*7);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {1, 0}, agg_descr), 7*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 2}, agg_descr), 9*5);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 0}, agg_descr), 5*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 3}, agg_descr), 5*3);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1, 2, 3}, agg_descr), 9*7*5);
+ }
+ }
+
+ Y_UNIT_TEST(AggregateMax) {
+ CH::RegisterAggregates();
+
+ auto block = CH::makeTestBlock(1000);
+
+ for (int i = 0; i < block->num_columns(); ++i) {
+ auto type = block->column(i)->type();
+ auto agg_descr = CH::MakeMinMaxAnyDescription("ch.max", type, i);
+
+ UNIT_ASSERT(agg_descr.function);
+ UNIT_ASSERT_VALUES_EQUAL(agg_descr.arguments.size(), 1);
+
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1}, agg_descr), 9*7);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {1, 0}, agg_descr), 7*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 2}, agg_descr), 9*5);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 0}, agg_descr), 5*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 3}, agg_descr), 5*3);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1, 2, 3}, agg_descr), 9*7*5);
+ }
+ }
+
+ Y_UNIT_TEST(AggregateAny) {
+ CH::RegisterAggregates();
+
+ auto block = CH::makeTestBlock(1000);
+
+ for (int i = 0; i < block->num_columns(); ++i) {
+ auto type = block->column(i)->type();
+ auto agg_descr = CH::MakeMinMaxAnyDescription("ch.any", type, i);
+
+ UNIT_ASSERT(agg_descr.function);
+ UNIT_ASSERT_VALUES_EQUAL(agg_descr.arguments.size(), 1);
+
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1}, agg_descr), 9*7);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {1, 0}, agg_descr), 7*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 2}, agg_descr), 9*5);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 0}, agg_descr), 5*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 3}, agg_descr), 5*3);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1, 2, 3}, agg_descr), 9*7*5);
+ }
+ }
+
+ Y_UNIT_TEST(AggregateSum) {
+ CH::RegisterAggregates();
+
+ auto block = CH::makeTestBlock(1000);
+
+ for (int i = 0; i < 2; ++i) {
+ auto type = block->column(i)->type();
+ auto agg_descr = CH::MakeSumDescription(type, i);
+
+ UNIT_ASSERT(agg_descr.function);
+ UNIT_ASSERT_VALUES_EQUAL(agg_descr.arguments.size(), 1);
+
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1}, agg_descr), 9*7);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {1, 0}, agg_descr), 7*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 2}, agg_descr), 9*5);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 0}, agg_descr), 5*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 3}, agg_descr), 5*3);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1, 2, 3}, agg_descr), 9*7*5);
+ }
+ }
+
+ Y_UNIT_TEST(AggregateAvg) {
+ CH::RegisterAggregates();
+
+ auto block = CH::makeTestBlock(1000);
+
+ for (int i = 0; i < 2; ++i) {
+ auto type = block->column(i)->type();
+ auto agg_descr = CH::MakeAvgDescription(type, i);
+
+ UNIT_ASSERT(agg_descr.function);
+ UNIT_ASSERT_VALUES_EQUAL(agg_descr.arguments.size(), 1);
+
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1}, agg_descr), 9*7);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {1, 0}, agg_descr), 7*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 2}, agg_descr), 9*5);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 0}, agg_descr), 5*9);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {2, 3}, agg_descr), 5*3);
+ UNIT_ASSERT_VALUES_EQUAL(CH::TestAggregate(block, {0, 1, 2, 3}, agg_descr), 9*7*5);
+ }
+ }
+}