Using GatherUtils (experimental) [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-07-24 08:21:17 +03:00 committed by alexey-milovidov
parent 366ad1595e
commit 0c8e082954
2 changed files with 69 additions and 270 deletions

View File

@ -48,11 +48,6 @@ struct EmptyImpl
{
}
static void constant(const std::string & data, UInt8 & res)
{
res = negative ^ (data.empty());
}
static void array(const ColumnString::Offsets_t & offsets, PaddedPODArray<UInt8> & res)
{
size_t size = offsets.size();
@ -63,11 +58,6 @@ struct EmptyImpl
prev_offset = offsets[i];
}
}
static void constant_array(const Array & data, UInt8 & res)
{
res = negative ^ (data.empty());
}
};
@ -91,22 +81,12 @@ struct LengthImpl
{
}
static void constant(const std::string & data, UInt64 & res)
{
res = data.size();
}
static void array(const ColumnString::Offsets_t & offsets, PaddedPODArray<UInt64> & res)
{
size_t size = offsets.size();
for (size_t i = 0; i < size; ++i)
res[i] = i == 0 ? (offsets[i]) : (offsets[i] - offsets[i - 1]);
}
static void constant_array(const Array & data, UInt64 & res)
{
res = data.size();
}
};
@ -143,20 +123,10 @@ struct LengthUTF8Impl
}
}
static void constant(const std::string & data, UInt64 & res)
{
res = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(data.data()), data.size());
}
static void array(const ColumnString::Offsets_t & offsets, PaddedPODArray<UInt64> & res)
{
throw Exception("Cannot apply function lengthUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static void constant_array(const Array & data, UInt64 & res)
{
throw Exception("Cannot apply function lengthUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
@ -179,14 +149,6 @@ struct LowerUpperImpl
array(data.data(), data.data() + data.size(), res_data.data());
}
static void constant(const std::string & data, std::string & res_data)
{
res_data.resize(data.size());
array(reinterpret_cast<const UInt8 *>(data.data()),
reinterpret_cast<const UInt8 *>(data.data() + data.size()),
reinterpret_cast<UInt8 *>(&res_data[0]));
}
private:
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
{
@ -260,13 +222,6 @@ struct ReverseImpl
for (size_t j = i * n; j < (i + 1) * n; ++j)
res_data[j] = data[(i * 2 + 1) * n - j - 1];
}
static void constant(const std::string & data, std::string & res_data)
{
res_data.resize(data.size());
for (size_t j = 0; j < data.size(); ++j)
res_data[j] = data[data.size() - j - 1];
}
};
@ -322,36 +277,6 @@ struct ReverseUTF8Impl
{
throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN);
}
static void constant(const std::string & data, std::string & res_data)
{
res_data.resize(data.size());
size_t j = 0;
while (j < data.size())
{
if (static_cast<unsigned char>(data[j]) < 0xBF)
{
res_data[data.size() - 1 - j] = data[j];
j += 1;
}
else if (static_cast<unsigned char>(data[j]) < 0xE0)
{
memcpy(&res_data[data.size() - 1 - j - 1], &data[j], 2);
j += 2;
}
else if (static_cast<unsigned char>(data[j]) < 0xF0)
{
memcpy(&res_data[data.size() - 1 - j - 2], &data[j], 3);
j += 3;
}
else
{
res_data[data.size() - 1 - j] = data[j];
j += 1;
}
}
}
};
@ -575,14 +500,6 @@ struct SubstringImpl
res_offsets[i] = res_offset;
}
}
static void constant(const std::string & data, size_t start, size_t length, std::string & res_data)
{
if (start + length > data.size() + 1)
throw Exception("Index out of bound for function substring of fixed size value", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
res_data = data.substr(start - 1, length);
}
};
@ -660,41 +577,6 @@ struct SubstringUTF8Impl
{
throw Exception("Cannot apply function substringUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN);
}
static void constant(const std::string & data, size_t start, size_t length, std::string & res_data)
{
if (start + length > data.size() + 1)
throw Exception("Index out of bound for function substring of constant value", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
ColumnString::Offset_t j = 0;
ColumnString::Offset_t pos = 1;
ColumnString::Offset_t bytes_start = 0;
ColumnString::Offset_t bytes_length = 0;
while (j < data.size())
{
if (pos == start)
bytes_start = j + 1;
if (static_cast<unsigned char>(data[j]) < 0xBF)
j += 1;
else if (static_cast<unsigned char>(data[j]) < 0xE0)
j += 2;
else if (static_cast<unsigned char>(data[j]) < 0xF0)
j += 3;
else
j += 1;
if (pos >= start && pos < start + length)
bytes_length = j + 1 - bytes_start;
else if (pos >= start + length)
break;
++pos;
}
if (bytes_start != 0)
res_data = data.substr(bytes_start - 1, bytes_length);
}
};
@ -903,75 +785,6 @@ public:
}
private:
enum class InstructionType : UInt8
{
COPY_STRING,
COPY_FIXED_STRING,
COPY_CONST_STRING,
COPY_CONST_FIXED_STRING,
};
/// column pointer augmented with offset (current offset String/FixedString, unused for Const<String>)
using ColumnAndOffset = std::pair<const IColumn *, IColumn::Offset_t>;
/// InstructionType is being stored to allow using static_cast safely
using Instruction = std::pair<InstructionType, ColumnAndOffset>;
using Instructions = std::vector<Instruction>;
/** calculate total length of resulting strings (without terminating nulls), determine whether all input
* strings are constant, assemble instructions
*/
Instructions getInstructions(const Block & block, const ColumnNumbers & arguments, size_t & out_length, bool & out_const)
{
Instructions result{};
result.reserve(arguments.size());
out_length = 0;
out_const = true;
size_t rows{};
for (const auto arg_pos : arguments)
{
const auto column = block.getByPosition(arg_pos).column.get();
if (const auto col = checkAndGetColumn<ColumnString>(column))
{
/** ColumnString stores strings with terminating null character
* which should not be copied, therefore the decrease of total size by
* the number of terminating nulls
*/
rows = col->size();
out_length += col->getChars().size() - col->getOffsets().size();
out_const = false;
result.emplace_back(InstructionType::COPY_STRING, ColumnAndOffset{col, 0});
}
else if (const auto col = checkAndGetColumn<ColumnFixedString>(column))
{
rows = col->size();
out_length += col->getChars().size();
out_const = false;
result.emplace_back(InstructionType::COPY_FIXED_STRING, ColumnAndOffset{col, 0});
}
else if (const auto col = checkAndGetColumnConstStringOrFixedString(column))
{
rows = col->size();
out_length += col->getValue<String>().size() * col->size();
out_const = out_const && true;
result.emplace_back(InstructionType::COPY_CONST_STRING, ColumnAndOffset{col, 0});
}
else
throw Exception(
"Illegal column " + column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
}
if (out_const && rows)
out_length /= rows;
return result;
}
void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result)
{
const IColumn * c0 = block.getByPosition(arguments[0]).column.get();
@ -986,7 +799,6 @@ private:
{
auto c_res = std::make_shared<ColumnString>();
block.getByPosition(result).column = c_res;
if (c0_string && c1_string)
concat(StringSource(*c0_string), StringSource(*c1_string), StringSink(*c_res, c0->size()));
@ -1010,87 +822,22 @@ private:
+ " of arguments of function "
+ getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = c_res;
}
}
void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result)
{
const auto size = block.rows();
size_t result_length{};
bool result_is_const{};
auto instrs = getInstructions(block, arguments, result_length, result_is_const);
size_t num_sources = arguments.size();
StringSources sources(num_sources);
if (result_is_const)
{
String res;
res.reserve(result_length);
for (size_t i = 0; i < num_sources; ++i)
sources[i] = createDynamicStringSource(*block.getByPosition(arguments[i]).column);
for (const auto & instr : instrs)
res += static_cast<const ColumnConst *>(instr.second.first)->getValue<String>();
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(size, res);
}
else
{
const auto out = std::make_shared<ColumnString>();
block.getByPosition(result).column = out;
auto & out_data = out->getChars();
out_data.resize(result_length + size);
auto & out_offsets = out->getOffsets();
out_offsets.resize(size);
size_t out_offset{};
for (const auto row : ext::range(0, size))
{
for (auto & instr : instrs)
{
switch (instr.first)
{
case InstructionType::COPY_STRING:
{
auto & in_offset = instr.second.second;
const auto col = static_cast<const ColumnString *>(instr.second.first);
const auto offset = col->getOffsets()[row];
const auto length = offset - in_offset - 1;
memcpySmallAllowReadWriteOverflow15(&out_data[out_offset], &col->getChars()[in_offset], length);
out_offset += length;
in_offset = offset;
break;
}
case InstructionType::COPY_FIXED_STRING:
{
auto & in_offset = instr.second.second;
const auto col = static_cast<const ColumnFixedString *>(instr.second.first);
const auto length = col->getN();
memcpySmallAllowReadWriteOverflow15(&out_data[out_offset], &col->getChars()[in_offset], length);
out_offset += length;
in_offset += length;
break;
}
case InstructionType::COPY_CONST_STRING:
{
const auto col = static_cast<const ColumnConst *>(instr.second.first);
const auto & data = col->getValue<String>();
const auto length = data.size();
memcpy(&out_data[out_offset], data.data(), length);
out_offset += length;
break;
}
default:
throw Exception("Unknown InstructionType during execution of function 'concat'", ErrorCodes::LOGICAL_ERROR);
}
}
out_data[out_offset] = '\0';
out_offsets[row] = ++out_offset;
}
}
auto c_res = std::make_shared<ColumnString>();
concat(sources, StringSink(*c_res, block.rows()));
block.getByPosition(result).column = c_res;
}
};
@ -1115,6 +862,9 @@ public:
return 3;
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!checkDataType<DataTypeString>(&*arguments[0]) && !checkDataType<DataTypeFixedString>(&*arguments[0]))
@ -1167,13 +917,6 @@ public:
block.getByPosition(result).column = col_res;
Impl::vector_fixed(col->getChars(), col->getN(), start, length, col_res->getChars(), col_res->getOffsets());
}
else if (const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(&*column_string))
{
String res;
Impl::constant(col->getValue<String>(), start, length, res);
auto col_res = DataTypeString().createConstColumn(col->size(), res);
block.getByPosition(result).column = col_res;
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),

View File

@ -6,6 +6,8 @@
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h>
#include <Functions/FunctionHelpers.h>
#include <Common/typeid_cast.h>
#include <Common/memcpySmall.h>
@ -14,11 +16,12 @@
* copy ranges from one or more columns to another column.
*
* Example:
* - concatenation of strings and arrays (concat, arrayStringConcat);
* - concatenation of strings and arrays (concat);
* - extracting slices and elements of strings and arrays (substring, arraySlice, arrayElement);
* - creating arrays from several columns ([x, y]);
* - conditional selecting from several string or array columns (if, multiIf);
* - push and pop elements from array front or back (arrayPushBack, etc);
* - splitting strings into arrays and joining arrays back;
* - conversion of numeric arrays between different types (CAST);
* - formatting strings (format).
*
@ -374,6 +377,45 @@ struct FixedStringSink
};
struct IStringSource
{
using Slice = NumericArraySlice<UInt8>;
virtual void next() = 0;
virtual bool isEnd() const = 0;
virtual Slice getWhole() const = 0;
virtual ~IStringSource() {}
};
template <typename Impl>
struct DynamicStringSource : 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(); };
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;
@ -525,4 +567,18 @@ void concat(SourceA && src_a, SourceB && src_b, Sink && sink)
}
}
template <typename Sink>
void concat(StringSources & sources, Sink && sink)
{
while (!sink.isEnd())
{
for (auto & source : sources)
{
writeSlice(source->getWhole(), sink);
source->next();
}
sink.next();
}
}
}