提交 ce5e329d 编写于 作者: A Alexey Milovidov

Using GatherUtils: preparation [#CLICKHOUSE-2].

上级 77bb8830
......@@ -7,6 +7,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/NumberTraits.h>
......@@ -450,47 +451,6 @@ struct FixedStringSink
};
struct IStringSource
{
using Slice = NumericArraySlice<UInt8>;
virtual void next() = 0;
virtual bool isEnd() const = 0;
virtual size_t getSizeForReserve() const = 0;
virtual Slice getWhole() const = 0;
virtual ~IStringSource() {}
};
template <typename Impl>
struct DynamicStringSource final : IStringSource
{
Impl impl;
DynamicStringSource(const IColumn & col) : impl(static_cast<const typename Impl::Column &>(col)) {}
void next() override { impl.next(); }
bool isEnd() const override { return impl.isEnd(); }
size_t getSizeForReserve() const override { return impl.getSizeForReserve(); }
Slice getWhole() const override { return impl.getWhole(); }
};
inline std::unique_ptr<IStringSource> createDynamicStringSource(const IColumn & col)
{
if (checkColumn<ColumnString>(&col))
return std::make_unique<DynamicStringSource<StringSource>>(col);
if (checkColumn<ColumnFixedString>(&col))
return std::make_unique<DynamicStringSource<FixedStringSource>>(col);
if (checkColumnConst<ColumnString>(&col))
return std::make_unique<DynamicStringSource<ConstSource<StringSource>>>(col);
if (checkColumnConst<ColumnFixedString>(&col))
return std::make_unique<DynamicStringSource<ConstSource<FixedStringSource>>>(col);
throw Exception("Unexpected type of string column: " + col.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
using StringSources = std::vector<std::unique_ptr<IStringSource>>;
struct GenericArraySlice
{
const IColumn * elements;
......@@ -695,6 +655,119 @@ struct NumericSink
};
struct IStringSource
{
using Slice = NumericArraySlice<UInt8>;
virtual void next() = 0;
virtual bool isEnd() const = 0;
virtual size_t getSizeForReserve() const = 0;
virtual Slice getWhole() const = 0;
virtual ~IStringSource() {}
};
template <typename Impl>
struct DynamicStringSource final : IStringSource
{
Impl impl;
DynamicStringSource(const IColumn & col) : impl(static_cast<const typename Impl::Column &>(col)) {}
void next() override { impl.next(); }
bool isEnd() const override { return impl.isEnd(); }
size_t getSizeForReserve() const override { return impl.getSizeForReserve(); }
Slice getWhole() const override { return impl.getWhole(); }
};
inline std::unique_ptr<IStringSource> createDynamicStringSource(const IColumn & col)
{
if (checkColumn<ColumnString>(&col))
return std::make_unique<DynamicStringSource<StringSource>>(col);
if (checkColumn<ColumnFixedString>(&col))
return std::make_unique<DynamicStringSource<FixedStringSource>>(col);
if (checkColumnConst<ColumnString>(&col))
return std::make_unique<DynamicStringSource<ConstSource<StringSource>>>(col);
if (checkColumnConst<ColumnFixedString>(&col))
return std::make_unique<DynamicStringSource<ConstSource<FixedStringSource>>>(col);
throw Exception("Unexpected type of string column: " + col.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
using StringSources = std::vector<std::unique_ptr<IStringSource>>;
enum class NumericType
{
UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64
};
struct DynamicNumericSlice
{
NumericType type;
const void * data;
};
struct INumericSource
{
using Slice = DynamicNumericSlice;
virtual void next() = 0;
virtual bool isEnd() const = 0;
virtual size_t getSizeForReserve() const = 0;
virtual Slice getWhole() const = 0;
virtual ~INumericSource() {}
};
template <typename Impl>
struct DynamicNumericSource final : INumericSource
{
Impl impl;
NumericType type;
DynamicNumericSource(const IColumn & col, NumericType type_) : impl(static_cast<const typename Impl::Column &>(col)), type(type_)
{
}
void next() override { impl.next(); }
bool isEnd() const override { return impl.isEnd(); }
size_t getSizeForReserve() const override { return impl.getSizeForReserve(); }
Slice getWhole() const override
{
return {type, impl.getWhole()};
}
};
inline std::unique_ptr<INumericSource> createDynamicNumericSource(const IColumn & col)
{
if (checkColumn<ColumnUInt8>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<UInt8>>>(col, NumericType::UInt8);
else if (checkColumn<ColumnUInt16>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<UInt16>>>(col, NumericType::UInt16);
else if (checkColumn<ColumnUInt32>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<UInt32>>>(col, NumericType::UInt32);
else if (checkColumn<ColumnUInt64>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<UInt64>>>(col, NumericType::UInt64);
else if (checkColumn<ColumnInt8>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<Int8>>>(col, NumericType::Int8);
else if (checkColumn<ColumnInt16>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<Int16>>>(col, NumericType::Int16);
else if (checkColumn<ColumnInt32>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<Int32>>>(col, NumericType::Int32);
else if (checkColumn<ColumnInt64>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<Int64>>>(col, NumericType::Int64);
else if (checkColumn<ColumnFloat32>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<Float32>>>(col, NumericType::Float32);
else if (checkColumn<ColumnFloat64>(&col)) return std::make_unique<DynamicNumericSource<NumericSource<Float64>>>(col, NumericType::Float64);
else if (checkColumnConst<ColumnUInt8>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<UInt8>>>>(col, NumericType::UInt8);
else if (checkColumnConst<ColumnUInt16>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<UInt16>>>>(col, NumericType::UInt16);
else if (checkColumnConst<ColumnUInt32>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<UInt32>>>>(col, NumericType::UInt32);
else if (checkColumnConst<ColumnUInt64>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<UInt64>>>>(col, NumericType::UInt64);
else if (checkColumnConst<ColumnInt8>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<Int8>>>>(col, NumericType::Int8);
else if (checkColumnConst<ColumnInt16>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<Int16>>>>(col, NumericType::Int16);
else if (checkColumnConst<ColumnInt32>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<Int32>>>>(col, NumericType::Int32);
else if (checkColumnConst<ColumnInt64>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<Int64>>>>(col, NumericType::Int64);
else if (checkColumnConst<ColumnFloat32>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<Float32>>>>(col, NumericType::Float32);
else if (checkColumnConst<ColumnFloat64>(&col)) return std::make_unique<DynamicNumericSource<ConstSource<NumericSource<Float64>>>>(col, NumericType::Float64);
else
throw Exception("Unexpected type of numeric column: " + col.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
using NumericSources = std::vector<std::unique_ptr<INumericSource>>;
/// Methods to copy Slice to Sink, overloaded for various combinations of types.
template <typename T>
......@@ -741,6 +814,24 @@ void writeSlice(const NumericSlice<T> & slice, NumericSink<U> & sink)
*sink.pos = *slice;
}
template <typename U>
void writeSlice(const DynamicNumericSlice & slice, NumericSink<U> & sink)
{
switch (slice.type)
{
case NumericType::UInt8: *sink.pos = *static_cast<const UInt8 *>(slice.data); break;
case NumericType::UInt16: *sink.pos = *static_cast<const UInt16 *>(slice.data); break;
case NumericType::UInt32: *sink.pos = *static_cast<const UInt32 *>(slice.data); break;
case NumericType::UInt64: *sink.pos = *static_cast<const UInt64 *>(slice.data); break;
case NumericType::Int8: *sink.pos = *static_cast<const Int8 *>(slice.data); break;
case NumericType::Int16: *sink.pos = *static_cast<const Int16 *>(slice.data); break;
case NumericType::Int32: *sink.pos = *static_cast<const Int32 *>(slice.data); break;
case NumericType::Int64: *sink.pos = *static_cast<const Int64 *>(slice.data); break;
case NumericType::Float32: *sink.pos = *static_cast<const Float32 *>(slice.data); break;
case NumericType::Float64: *sink.pos = *static_cast<const Float64 *>(slice.data); break;
}
}
/// Algorithms
......@@ -892,4 +983,34 @@ void NO_INLINE conditional(SourceA && src_a, SourceB && src_b, Sink && sink, con
}
}
template <typename Sources, typename Sink>
void NO_INLINE conditionalMulti(Sources && sources, Sink && sink, const PaddedPODArray<UInt8> * conditions)
{
size_t max_reserve_size = 0;
for (const auto & source : sources)
max_reserve_size = std::max(max_reserve_size, source->getSizeForReserve());
sink.reserve(max_reserve_size);
size_t rows = conditions[0].size();
size_t num_conditions = sources.size() - 1;
for (size_t row_num = 0; row_num < rows; ++row_num)
{
size_t condition_index = 0;
while (true)
{
if (condition_index == num_conditions || conditions[condition_index][row_num])
{
writeSlice(sources[condition_index]->getWhole(), sink);
break;
}
++condition_index;
}
for (const auto & source : sources)
source->next();
sink.next();
}
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册