Merge branch 'master' of https://github.com/yandex/ClickHouse into CLICKHOUSE-2720

This commit is contained in:
Ivan Blinkov 2018-05-02 01:04:12 +03:00
commit efc68b0e16
80 changed files with 2137 additions and 1008 deletions

View File

@ -1,8 +1,6 @@
# ClickHouse
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
🎤🥂 **ClickHouse Meetup in [Sunnyvale](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/248898966/) & [San Francisco](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/249162518/), April 23-27** 🍰🔥🐻
Learn more about ClickHouse at [https://clickhouse.yandex/](https://clickhouse.yandex/)
[![Build Status](https://travis-ci.org/yandex/ClickHouse.svg?branch=master)](https://travis-ci.org/yandex/ClickHouse)

View File

@ -26,6 +26,11 @@
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeFunction.h>
#include <algorithm>
#include <sstream>
#include <unordered_map>
#include <vector>
#include <memory>
namespace DB
@ -251,7 +256,7 @@ void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantIn
size_t result_position = argument_numbers.size();
block_with_constants.insert({nullptr, expression_info.data_type, column_name});
function_ptr->execute(block_with_constants, argument_numbers, result_position);
function_ptr->execute(block_with_constants, argument_numbers, result_position, 1);
const auto & result_column = block_with_constants.getByPosition(result_position).column;
if (result_column->isColumnConst())

View File

@ -91,6 +91,11 @@ public:
return data->getInt(0);
}
UInt8 getBoolRepresentation(size_t) const override
{
return data->getBoolRepresentation(0);
}
bool isNullAt(size_t) const override
{
return data->isNullAt(0);

View File

@ -194,7 +194,7 @@ ColumnWithTypeAndName ColumnFunction::reduce() const
for (size_t i = 0; i < captured_columns.size(); ++i)
arguments[i] = i;
function->execute(block, arguments, captured_columns.size());
function->execute(block, arguments, captured_columns.size(), size_);
return block.getByPosition(captured_columns.size());
}

View File

@ -46,6 +46,7 @@ public:
bool isNullAt(size_t n) const override { return static_cast<const ColumnUInt8 &>(*null_map).getData()[n] != 0;}
Field operator[](size_t n) const override;
void get(size_t n, Field & res) const override;
UInt8 getBoolRepresentation(size_t n) const override { return isNullAt(n) ? 0 : nested_column->getBoolRepresentation(n); }
UInt64 get64(size_t n) const override { return nested_column->get64(n); }
StringRef getDataAt(size_t n) const override;
void insertData(const char * pos, size_t length) override;

View File

@ -231,6 +231,11 @@ public:
return UInt64(data[n]);
}
UInt8 getBoolRepresentation(size_t n) const override
{
return UInt8(!!data[n]);
}
Int64 getInt(size_t n) const override
{
return Int64(data[n]);

View File

@ -95,6 +95,11 @@ public:
throw Exception("Method getUInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual UInt8 getBoolRepresentation(size_t /*n*/) const
{
throw Exception("Method getBoolRepresentation is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual Int64 getInt(size_t /*n*/) const
{
throw Exception("Method getInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);

View File

@ -378,6 +378,9 @@ namespace ErrorCodes
extern const int POCO_EXCEPTION = 1000;
extern const int STD_EXCEPTION = 1001;
extern const int UNKNOWN_EXCEPTION = 1002;
extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001;
extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002;
}
}

View File

@ -7,8 +7,8 @@ namespace DB
{
CSVRowOutputStream::CSVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_, bool with_types_)
: ostr(ostr_), sample(sample_), with_names(with_names_), with_types(with_types_)
CSVRowOutputStream::CSVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const char delimiter_, bool with_names_, bool with_types_)
: ostr(ostr_), sample(sample_), delimiter(delimiter_), with_names(with_names_), with_types(with_types_)
{
size_t columns = sample.columns();
data_types.resize(columns);
@ -32,7 +32,7 @@ void CSVRowOutputStream::writePrefix()
for (size_t i = 0; i < columns; ++i)
{
writeCSVString(sample.safeGetByPosition(i).name, ostr);
writeChar(i == columns - 1 ? '\n' : ',', ostr);
writeChar(i == columns - 1 ? '\n' : delimiter, ostr);
}
}
@ -41,7 +41,7 @@ void CSVRowOutputStream::writePrefix()
for (size_t i = 0; i < columns; ++i)
{
writeCSVString(sample.safeGetByPosition(i).type->getName(), ostr);
writeChar(i == columns - 1 ? '\n' : ',', ostr);
writeChar(i == columns - 1 ? '\n' : delimiter, ostr);
}
}
}
@ -55,7 +55,7 @@ void CSVRowOutputStream::writeField(const IColumn & column, const IDataType & ty
void CSVRowOutputStream::writeFieldDelimiter()
{
writeChar(',', ostr);
writeChar(delimiter, ostr);
}

View File

@ -19,7 +19,7 @@ public:
/** with_names - output in the first line a header with column names
* with_types - output in the next line header with the names of the types
*/
CSVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_ = false, bool with_types_ = false);
CSVRowOutputStream(WriteBuffer & ostr_, const Block & sample_, const char delimiter_, bool with_names_ = false, bool with_types_ = false);
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writeFieldDelimiter() override;
@ -44,6 +44,7 @@ protected:
WriteBuffer & ostr;
const Block sample;
const char delimiter;
bool with_names;
bool with_types;
DataTypes data_types;

View File

@ -81,13 +81,12 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu
{
return wrap_row_stream(std::make_shared<ValuesRowInputStream>(buf, sample, context, settings.input_format_values_interpret_expressions));
}
else if (name == "CSV")
else if (name == "CSV" || name == "CSVWithNames")
{
return wrap_row_stream(std::make_shared<CSVRowInputStream>(buf, sample, ','));
}
else if (name == "CSVWithNames")
{
return wrap_row_stream(std::make_shared<CSVRowInputStream>(buf, sample, ',', true));
char csv_delimiter = settings.format_csv_delimiter;
bool with_names = name == "CSVWithNames";
return wrap_row_stream(std::make_shared<CSVRowInputStream>(buf, sample, csv_delimiter, with_names));
}
else if (name == "TSKV")
{
@ -152,10 +151,13 @@ static BlockOutputStreamPtr getOutputImpl(const String & name, WriteBuffer & buf
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<TabSeparatedRowOutputStream>(buf, sample, true, true), sample);
else if (name == "TabSeparatedRaw" || name == "TSVRaw")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<TabSeparatedRawRowOutputStream>(buf, sample), sample);
else if (name == "CSV")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<CSVRowOutputStream>(buf, sample), sample);
else if (name == "CSVWithNames")
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<CSVRowOutputStream>(buf, sample, true), sample);
else if (name == "CSV" || name == "CSVWithNames")
{
char csv_delimiter = settings.format_csv_delimiter;
bool with_names = name == "CSVWithNames";
return std::make_shared<BlockOutputStreamFromRowOutputStream>(std::make_shared<CSVRowOutputStream>(buf, sample, csv_delimiter, with_names), sample);
}
else if (name == "Pretty")
return std::make_shared<PrettyBlockOutputStream>(buf, sample, false, settings.output_format_pretty_max_rows, context);
else if (name == "PrettyCompact")

View File

@ -194,9 +194,9 @@ void DataTypeFixedString::serializeTextCSV(const IColumn & column, size_t row_nu
}
void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char /*delimiter*/) const
void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const
{
read(*this, column, [&istr](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr); });
read(*this, column, [&istr, delimiter](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr, delimiter); });
}

View File

@ -285,9 +285,9 @@ void DataTypeString::serializeTextCSV(const IColumn & column, size_t row_num, Wr
}
void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char /*delimiter*/) const
void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const
{
read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr); });
read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr, delimiter); });
}

View File

@ -68,6 +68,13 @@ generate_function_register(Array
FunctionArrayResize
)
generate_function_register(Projection
FunctionOneOrZero
FunctionProject
FunctionBuildProjectionComposition
FunctionRestoreProjection
)
add_headers_and_sources(clickhouse_functions .)
add_headers_and_sources(clickhouse_functions ./GatherUtils)

View File

@ -44,7 +44,6 @@ Columns convertConstTupleToConstantElements(const ColumnConst & column)
static Block createBlockWithNestedColumnsImpl(const Block & block, const std::unordered_set<size_t> & args)
{
Block res;
size_t rows = block.rows();
size_t columns = block.columns();
for (size_t i = 0; i < columns; ++i)
@ -70,7 +69,7 @@ static Block createBlockWithNestedColumnsImpl(const Block & block, const std::un
const auto & nested_col = static_cast<const ColumnNullable &>(
static_cast<const ColumnConst &>(*col.column).getDataColumn()).getNestedColumnPtr();
res.insert({ ColumnConst::create(nested_col, rows), nested_type, col.name});
res.insert({ ColumnConst::create(nested_col, col.column->size()), nested_type, col.name});
}
else
throw Exception("Illegal column for DataTypeNullable", ErrorCodes::ILLEGAL_COLUMN);

View File

@ -64,9 +64,9 @@ struct BinaryOperationImplBase
c[i] = Op::template apply<ResultType>(a, b[i]);
}
static void constant_constant(A a, B b, ResultType & c)
static ResultType constant_constant(A a, B b)
{
c = Op::template apply<ResultType>(a, b);
return Op::template apply<ResultType>(a, b);
}
};
@ -476,27 +476,13 @@ struct IntExp10Impl
}
};
/// this one is just for convenience
template <bool B, typename T1, typename T2> using If = std::conditional_t<B, T1, T2>;
/// these ones for better semantics
template <typename T> using Then = T;
template <typename T> using Else = T;
/// Used to indicate undefined operation
struct InvalidType;
template <typename T>
struct DataTypeFromFieldType
{
using Type = DataTypeNumber<T>;
};
template <bool V, typename T> struct Case : std::bool_constant<V> { using type = T; };
template <>
struct DataTypeFromFieldType<NumberTraits::Error>
{
using Type = InvalidType;
};
/// Switch<Case<C0, T0>, ...> -- select the first Ti for which Ci is true; InvalidType if none.
template <typename... Ts> using Switch = typename std::disjunction<Ts..., Case<true, InvalidType>>::type;
template <typename DataType> constexpr bool IsIntegral = false;
template <> constexpr bool IsIntegral<DataTypeUInt8> = true;
@ -512,270 +498,74 @@ template <typename DataType> constexpr bool IsDateOrDateTime = false;
template <> constexpr bool IsDateOrDateTime<DataTypeDate> = true;
template <> constexpr bool IsDateOrDateTime<DataTypeDateTime> = true;
/** Returns appropriate result type for binary operator on dates (or datetimes):
* Date + Integral -> Date
* Integral + Date -> Date
* Date - Date -> Int32
* Date - Integral -> Date
* least(Date, Date) -> Date
* greatest(Date, Date) -> Date
* All other operations are not defined and return InvalidType, operations on
* distinct date types are also undefined (e.g. DataTypeDate - DataTypeDateTime)
*/
template <typename T> using DataTypeFromFieldType = std::conditional_t<std::is_same_v<T, NumberTraits::Error>, InvalidType, DataTypeNumber<T>>;
template <template <typename, typename> class Operation, typename LeftDataType, typename RightDataType>
struct DateBinaryOperationTraits
struct BinaryOperationTraits
{
using T0 = typename LeftDataType::FieldType;
using T1 = typename RightDataType::FieldType;
using Op = Operation<T0, T1>;
using ResultDataType =
If<std::is_same_v<Op, PlusImpl<T0, T1>>,
Then<
If<IsDateOrDateTime<LeftDataType> && IsIntegral<RightDataType>,
Then<LeftDataType>,
Else<
If<IsIntegral<LeftDataType> && IsDateOrDateTime<RightDataType>,
Then<RightDataType>,
Else<InvalidType>>>>>,
Else<
If<std::is_same_v<Op, MinusImpl<T0, T1>>,
Then<
If<IsDateOrDateTime<LeftDataType>,
Then<
If<std::is_same_v<LeftDataType, RightDataType>,
Then<DataTypeInt32>,
Else<
If<IsIntegral<RightDataType>,
Then<LeftDataType>,
Else<InvalidType>>>>>,
Else<InvalidType>>>,
Else<
If<std::is_same_v<T0, T1>
&& (std::is_same_v<Op, LeastImpl<T0, T1>> || std::is_same_v<Op, GreatestImpl<T0, T1>>),
Then<LeftDataType>,
Else<InvalidType>>>>>>;
/// Appropriate result type for binary operator on numeric types. "Date" can also mean
/// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid).
using ResultDataType = Switch<
/// number <op> number -> see corresponding impl
Case<!IsDateOrDateTime<LeftDataType> && !IsDateOrDateTime<RightDataType>,
DataTypeFromFieldType<typename Op::ResultType>>,
/// Date + Integral -> Date
/// Integral + Date -> Date
Case<std::is_same_v<Op, PlusImpl<T0, T1>>, Switch<
Case<IsIntegral<RightDataType>, LeftDataType>,
Case<IsIntegral<LeftDataType>, RightDataType>>>,
/// Date - Date -> Int32
/// Date - Integral -> Date
Case<std::is_same_v<Op, MinusImpl<T0, T1>>, Switch<
Case<std::is_same_v<LeftDataType, RightDataType>, DataTypeInt32>,
Case<IsDateOrDateTime<LeftDataType> && IsIntegral<RightDataType>, LeftDataType>>>,
/// least(Date, Date) -> Date
/// greatest(Date, Date) -> Date
Case<std::is_same_v<LeftDataType, RightDataType> && (std::is_same_v<Op, LeastImpl<T0, T1>> || std::is_same_v<Op, GreatestImpl<T0, T1>>),
LeftDataType>>;
};
/// Decides among date and numeric operations
template <template <typename, typename> class Operation, typename LeftDataType, typename RightDataType>
struct BinaryOperationTraits
template <typename... Ts, typename F>
static bool castTypeToEither(const IDataType * type, F && f)
{
using ResultDataType =
If<IsDateOrDateTime<LeftDataType> || IsDateOrDateTime<RightDataType>,
Then<
typename DateBinaryOperationTraits<
Operation, LeftDataType, RightDataType>::ResultDataType>,
Else<
typename DataTypeFromFieldType<
typename Operation<
typename LeftDataType::FieldType,
typename RightDataType::FieldType>::ResultType>::Type>>;
};
/// XXX can't use && here because gcc-7 complains about parentheses around && within ||
return ((typeid_cast<const Ts *>(type) ? f(*typeid_cast<const Ts *>(type)) : false) || ...);
}
template <template <typename, typename> class Op, typename Name>
class FunctionBinaryArithmetic : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBinaryArithmetic>(context); }
FunctionBinaryArithmetic(const Context & context) : context(context) {}
private:
const Context & context;
template <typename ResultDataType>
bool checkRightTypeImpl(DataTypePtr & type_res) const
template <typename F>
static bool castType(const IDataType * type, F && f)
{
/// Overload for InvalidType
if constexpr (std::is_same_v<ResultDataType, InvalidType>)
return false;
else
{
type_res = std::make_shared<ResultDataType>();
return true;
}
return castTypeToEither<
DataTypeUInt8,
DataTypeUInt16,
DataTypeUInt32,
DataTypeUInt64,
DataTypeInt8,
DataTypeInt16,
DataTypeInt32,
DataTypeInt64,
DataTypeFloat32,
DataTypeFloat64,
DataTypeDate,
DataTypeDateTime
>(type, std::forward<F>(f));
}
template <typename LeftDataType, typename RightDataType>
bool checkRightType(const DataTypes & arguments, DataTypePtr & type_res) const
template <typename F>
static bool castBothTypes(const IDataType * left, const IDataType * right, F && f)
{
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if (typeid_cast<const RightDataType *>(arguments[1].get()))
return checkRightTypeImpl<ResultDataType>(type_res);
return false;
}
template <typename T0>
bool checkLeftType(const DataTypes & arguments, DataTypePtr & type_res) const
{
if (typeid_cast<const T0 *>(arguments[0].get()))
{
if ( checkRightType<T0, DataTypeDate>(arguments, type_res)
|| checkRightType<T0, DataTypeDateTime>(arguments, type_res)
|| checkRightType<T0, DataTypeUInt8>(arguments, type_res)
|| checkRightType<T0, DataTypeUInt16>(arguments, type_res)
|| checkRightType<T0, DataTypeUInt32>(arguments, type_res)
|| checkRightType<T0, DataTypeUInt64>(arguments, type_res)
|| checkRightType<T0, DataTypeInt8>(arguments, type_res)
|| checkRightType<T0, DataTypeInt16>(arguments, type_res)
|| checkRightType<T0, DataTypeInt32>(arguments, type_res)
|| checkRightType<T0, DataTypeInt64>(arguments, type_res)
|| checkRightType<T0, DataTypeFloat32>(arguments, type_res)
|| checkRightType<T0, DataTypeFloat64>(arguments, type_res))
return true;
}
return false;
}
/// Overload for date operations
template <typename LeftDataType, typename RightDataType, typename ColumnType>
bool executeRightType(Block & block, const ColumnNumbers & arguments, const size_t result, const ColumnType * col_left)
{
if (!typeid_cast<const RightDataType *>(block.getByPosition(arguments[1]).type.get()))
return false;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
return executeRightTypeDispatch<LeftDataType, RightDataType, ResultDataType>(
block, arguments, result, col_left);
}
/// Overload for InvalidType
template <typename LeftDataType, typename RightDataType, typename ResultDataType, typename ColumnType>
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments,
[[maybe_unused]] const size_t result, [[maybe_unused]] const ColumnType * col_left)
{
if constexpr (std::is_same_v<ResultDataType, InvalidType>)
throw Exception("Types " + String(TypeName<typename LeftDataType::FieldType>::get())
+ " and " + String(TypeName<typename LeftDataType::FieldType>::get())
+ " are incompatible for function " + getName() + " or not upscaleable to common type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else
{
using T0 = typename LeftDataType::FieldType;
using T1 = typename RightDataType::FieldType;
using ResultType = typename ResultDataType::FieldType;
return executeRightTypeImpl<T0, T1, ResultType>(block, arguments, result, col_left);
}
}
/// ColumnVector overload
template <typename T0, typename T1, typename ResultType = typename Op<T0, T1>::ResultType>
bool executeRightTypeImpl(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
{
if (auto col_right = checkAndGetColumn<ColumnVector<T1>>(block.getByPosition(arguments[1]).column.get()))
{
auto col_res = ColumnVector<ResultType>::create();
auto & vec_res = col_res->getData();
vec_res.resize(col_left->getData().size());
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::vector_vector(col_left->getData(), col_right->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
}
else if (auto col_right = checkAndGetColumnConst<ColumnVector<T1>>(block.getByPosition(arguments[1]).column.get()))
{
auto col_res = ColumnVector<ResultType>::create();
auto & vec_res = col_res->getData();
vec_res.resize(col_left->getData().size());
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::vector_constant(col_left->getData(), col_right->template getValue<T1>(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
}
throw Exception("Logical error: unexpected type of column", ErrorCodes::LOGICAL_ERROR);
}
/// ColumnConst overload
template <typename T0, typename T1, typename ResultType = typename Op<T0, T1>::ResultType>
bool executeRightTypeImpl(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst * col_left)
{
if (auto col_right = checkAndGetColumn<ColumnVector<T1>>(block.getByPosition(arguments[1]).column.get()))
{
auto col_res = ColumnVector<ResultType>::create();
auto & vec_res = col_res->getData();
vec_res.resize(col_left->size());
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
}
else if (auto col_right = checkAndGetColumnConst<ColumnVector<T1>>(block.getByPosition(arguments[1]).column.get()))
{
ResultType res = 0;
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>(), res);
block.getByPosition(result).column = DataTypeNumber<ResultType>().createColumnConst(col_left->size(), toField(res));
return true;
}
return false;
}
template <typename LeftDataType>
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const size_t result)
{
if (!typeid_cast<const LeftDataType *>(block.getByPosition(arguments[0]).type.get()))
return false;
return executeLeftTypeImpl<LeftDataType>(block, arguments, result);
}
template <typename LeftDataType>
bool executeLeftTypeImpl(Block & block, const ColumnNumbers & arguments, const size_t result)
{
if (auto col_left = checkAndGetColumn<ColumnVector<typename LeftDataType::FieldType>>(block.getByPosition(arguments[0]).column.get()))
{
if ( executeRightType<LeftDataType, DataTypeDate>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeDateTime>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeUInt8>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeUInt16>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeUInt32>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeUInt64>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeInt8>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeInt16>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeInt32>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeInt64>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeFloat32>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeFloat64>(block, arguments, result, col_left))
return true;
else
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else if (auto col_left = checkAndGetColumnConst<ColumnVector<typename LeftDataType::FieldType>>(block.getByPosition(arguments[0]).column.get()))
{
if ( executeRightType<LeftDataType, DataTypeDate>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeDateTime>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeUInt8>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeUInt16>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeUInt32>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeUInt64>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeInt8>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeInt16>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeInt32>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeInt64>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeFloat32>(block, arguments, result, col_left)
|| executeRightType<LeftDataType, DataTypeFloat64>(block, arguments, result, col_left))
return true;
else
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return false;
return castType(left, [&](const auto & left) { return castType(right, [&](const auto & right) { return f(left, right); }); });
}
FunctionBuilderPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1) const
@ -820,6 +610,11 @@ private:
}
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBinaryArithmetic>(context); }
FunctionBinaryArithmetic(const Context & context) : context(context) {}
String getName() const override
{
return name;
@ -849,26 +644,25 @@ public:
}
DataTypePtr type_res;
if (!( checkLeftType<DataTypeDate>(arguments, type_res)
|| checkLeftType<DataTypeDateTime>(arguments, type_res)
|| checkLeftType<DataTypeUInt8>(arguments, type_res)
|| checkLeftType<DataTypeUInt16>(arguments, type_res)
|| checkLeftType<DataTypeUInt32>(arguments, type_res)
|| checkLeftType<DataTypeUInt64>(arguments, type_res)
|| checkLeftType<DataTypeInt8>(arguments, type_res)
|| checkLeftType<DataTypeInt16>(arguments, type_res)
|| checkLeftType<DataTypeInt32>(arguments, type_res)
|| checkLeftType<DataTypeInt64>(arguments, type_res)
|| checkLeftType<DataTypeFloat32>(arguments, type_res)
|| checkLeftType<DataTypeFloat64>(arguments, type_res)))
bool valid = castBothTypes(arguments[0].get(), arguments[1].get(), [&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
{
type_res = std::make_shared<ResultDataType>();
return true;
}
return false;
});
if (!valid)
throw Exception("Illegal types " + arguments[0]->getName() + " and " + arguments[1]->getName() + " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
if (auto function_builder = getFunctionForIntervalArithmetic(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type))
@ -887,27 +681,69 @@ public:
{new_block.getByPosition(new_arguments[0]), new_block.getByPosition(new_arguments[1])};
auto function = function_builder->build(new_arguments_with_type_and_name);
function->execute(new_block, new_arguments, result);
function->execute(new_block, new_arguments, result, input_rows_count);
block.getByPosition(result).column = new_block.getByPosition(result).column;
return;
}
if (!( executeLeftType<DataTypeDate>(block, arguments, result)
|| executeLeftType<DataTypeDateTime>(block, arguments, result)
|| executeLeftType<DataTypeUInt8>(block, arguments, result)
|| executeLeftType<DataTypeUInt16>(block, arguments, result)
|| executeLeftType<DataTypeUInt32>(block, arguments, result)
|| executeLeftType<DataTypeUInt64>(block, arguments, result)
|| executeLeftType<DataTypeInt8>(block, arguments, result)
|| executeLeftType<DataTypeInt16>(block, arguments, result)
|| executeLeftType<DataTypeInt32>(block, arguments, result)
|| executeLeftType<DataTypeInt64>(block, arguments, result)
|| executeLeftType<DataTypeFloat32>(block, arguments, result)
|| executeLeftType<DataTypeFloat64>(block, arguments, result)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
auto * left = block.getByPosition(arguments[0]).type.get();
auto * right = block.getByPosition(arguments[1]).type.get();
bool valid = castBothTypes(left, right, [&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
{
using T0 = typename LeftDataType::FieldType;
using T1 = typename RightDataType::FieldType;
using ResultType = typename ResultDataType::FieldType;
using OpImpl = BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>;
auto col_left_raw = block.getByPosition(arguments[0]).column.get();
auto col_right_raw = block.getByPosition(arguments[1]).column.get();
if (auto col_left = checkAndGetColumnConst<ColumnVector<T0>>(col_left_raw))
{
if (auto col_right = checkAndGetColumnConst<ColumnVector<T1>>(col_right_raw))
{
/// the only case with a non-vector result
auto res = OpImpl::constant_constant(col_left->template getValue<T0>(), col_right->template getValue<T1>());
block.getByPosition(result).column = ResultDataType().createColumnConst(col_left->size(), toField(res));
return true;
}
}
auto col_res = ColumnVector<ResultType>::create();
auto & vec_res = col_res->getData();
vec_res.resize(block.rows());
if (auto col_left = checkAndGetColumnConst<ColumnVector<T0>>(col_left_raw))
{
if (auto col_right = checkAndGetColumn<ColumnVector<T1>>(col_right_raw))
OpImpl::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res);
else
return false;
}
else if (auto col_left = checkAndGetColumn<ColumnVector<T0>>(col_left_raw))
{
if (auto col_right = checkAndGetColumn<ColumnVector<T1>>(col_right_raw))
OpImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res);
else if (auto col_right = checkAndGetColumnConst<ColumnVector<T1>>(col_right_raw))
OpImpl::vector_constant(col_left->getData(), col_right->template getValue<T1>(), vec_res);
else
return false;
}
else
{
return false;
}
block.getByPosition(result).column = std::move(col_res);
return true;
}
return false;
});
if (!valid)
throw Exception(getName() + "'s arguments do not match the expected data types", ErrorCodes::LOGICAL_ERROR);
}
};
@ -919,43 +755,27 @@ struct FunctionUnaryArithmeticMonotonicity;
template <template <typename> class Op, typename Name, bool is_injective>
class FunctionUnaryArithmetic : public IFunction
{
template <typename F>
static bool castType(const IDataType * type, F && f)
{
return castTypeToEither<
DataTypeUInt8,
DataTypeUInt16,
DataTypeUInt32,
DataTypeUInt64,
DataTypeInt8,
DataTypeInt16,
DataTypeInt32,
DataTypeInt64,
DataTypeFloat32,
DataTypeFloat64
>(type, std::forward<F>(f));
}
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionUnaryArithmetic>(); }
private:
template <typename T0>
bool checkType(const DataTypes & arguments, DataTypePtr & result) const
{
if (typeid_cast<const T0 *>(arguments[0].get()))
{
result = std::make_shared<DataTypeNumber<typename Op<typename T0::FieldType>::ResultType>>();
return true;
}
return false;
}
template <typename T0>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (const ColumnVector<T0> * col = checkAndGetColumn<ColumnVector<T0>>(block.getByPosition(arguments[0]).column.get()))
{
using ResultType = typename Op<T0>::ResultType;
auto col_res = ColumnVector<ResultType>::create();
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
UnaryOperationImpl<T0, Op<T0>>::vector(col->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
}
return false;
}
public:
String getName() const override
{
return name;
@ -969,38 +789,36 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
DataTypePtr result;
if (!( checkType<DataTypeUInt8>(arguments, result)
|| checkType<DataTypeUInt16>(arguments, result)
|| checkType<DataTypeUInt32>(arguments, result)
|| checkType<DataTypeUInt64>(arguments, result)
|| checkType<DataTypeInt8>(arguments, result)
|| checkType<DataTypeInt16>(arguments, result)
|| checkType<DataTypeInt32>(arguments, result)
|| checkType<DataTypeInt64>(arguments, result)
|| checkType<DataTypeFloat32>(arguments, result)
|| checkType<DataTypeFloat64>(arguments, result)))
bool valid = castType(arguments[0].get(), [&](const auto & type)
{
using T0 = typename std::decay_t<decltype(type)>::FieldType;
result = std::make_shared<DataTypeNumber<typename Op<T0>::ResultType>>();
return true;
});
if (!valid)
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return result;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)
|| executeType<UInt32>(block, arguments, result)
|| executeType<UInt64>(block, arguments, result)
|| executeType<Int8>(block, arguments, result)
|| executeType<Int16>(block, arguments, result)
|| executeType<Int32>(block, arguments, result)
|| executeType<Int64>(block, arguments, result)
|| executeType<Float32>(block, arguments, result)
|| executeType<Float64>(block, arguments, result)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
bool valid = castType(block.getByPosition(arguments[0]).type.get(), [&](const auto & type)
{
using T0 = typename std::decay_t<decltype(type)>::FieldType;
if (auto col = checkAndGetColumn<ColumnVector<T0>>(block.getByPosition(arguments[0]).column.get()))
{
auto col_res = ColumnVector<typename Op<T0>::ResultType>::create();
auto & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
UnaryOperationImpl<T0, Op<T0>>::vector(col->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
}
return false;
});
if (!valid)
throw Exception(getName() + "'s argument does not match the expected data type", ErrorCodes::LOGICAL_ERROR);
}
bool hasInformationAboutMonotonicity() const override
@ -1322,7 +1140,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block , const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto value_col = block.getByPosition(arguments.front()).column.get();

View File

@ -58,21 +58,21 @@ DataTypePtr FunctionArray::getReturnTypeImpl(const DataTypes & arguments) const
return std::make_shared<DataTypeArray>(getLeastSupertype(arguments));
}
void FunctionArray::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArray::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
size_t num_elements = arguments.size();
if (num_elements == 0)
{
/// We should return constant empty array.
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
const DataTypePtr & return_type = block.getByPosition(result).type;
const DataTypePtr & elem_type = static_cast<const DataTypeArray &>(*return_type).getNestedType();
size_t block_size = block.rows();
size_t block_size = input_rows_count;
/** If part of columns have not same type as common type of all elements of array,
* then convert them to common type.
@ -625,8 +625,9 @@ bool FunctionArrayElement::executeGeneric(Block & block, const ColumnNumbers & a
}
template <typename IndexType>
bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
ArrayImpl::NullMapBuilder & builder)
bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
size_t input_rows_count)
{
const ColumnArray * col_array = checkAndGetColumnConstData<ColumnArray>(block.getByPosition(arguments[0]).column.get());
@ -635,7 +636,7 @@ bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arg
auto res = block.getByPosition(result).type->createColumn();
size_t rows = block.rows();
size_t rows = input_rows_count;
const IColumn & array_elements = col_array->getData();
size_t array_size = array_elements.size();
@ -670,7 +671,7 @@ bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arg
template <typename IndexType>
bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers & arguments, size_t result,
ArrayImpl::NullMapBuilder & builder)
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count)
{
auto index = checkAndGetColumn<ColumnVector<IndexType>>(block.getByPosition(arguments[1]).column.get());
@ -692,7 +693,7 @@ bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers &
|| executeNumber<IndexType, Int64>(block, arguments, result, index_data, builder)
|| executeNumber<IndexType, Float32>(block, arguments, result, index_data, builder)
|| executeNumber<IndexType, Float64>(block, arguments, result, index_data, builder)
|| executeConst <IndexType>(block, arguments, result, index_data, builder)
|| executeConst<IndexType>(block, arguments, result, index_data, builder, input_rows_count)
|| executeString<IndexType>(block, arguments, result, index_data, builder)
|| executeGeneric<IndexType>(block, arguments, result, index_data, builder)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
@ -701,7 +702,7 @@ bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers &
return true;
}
bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arguments, size_t result)
bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
@ -747,7 +748,7 @@ bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arg
{block_of_temporary_results.getByPosition(i * 2 + 1).type, block_of_temporary_results.getByPosition(0).type});
block_of_temporary_results.insert(array_elements_of_tuple_section);
executeImpl(block_of_temporary_results, ColumnNumbers{i * 2 + 1, 0}, i * 2 + 2);
executeImpl(block_of_temporary_results, ColumnNumbers{i * 2 + 1, 0}, i * 2 + 2, input_rows_count);
result_tuple_columns.emplace_back(std::move(block_of_temporary_results.getByPosition(i * 2 + 2).column));
}
@ -774,7 +775,7 @@ DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments)
return array_type->getNestedType();
}
void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
/// Check nullability.
bool is_array_of_nullable = false;
@ -798,7 +799,7 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
if (!is_array_of_nullable)
{
ArrayImpl::NullMapBuilder builder;
perform(block, arguments, result, builder);
perform(block, arguments, result, builder, input_rows_count);
}
else
{
@ -841,7 +842,7 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
source_block =
{
{
ColumnConst::create(ColumnArray::create(nested_col, col_const_array->getOffsetsPtr()), block.rows()),
ColumnConst::create(ColumnArray::create(nested_col, col_const_array->getOffsetsPtr()), input_rows_count),
std::make_shared<DataTypeArray>(input_type),
""
},
@ -856,7 +857,7 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
builder.initSource(nullable_col.getNullMapData().data());
}
perform(source_block, {0, 1}, 2, builder);
perform(source_block, {0, 1}, 2, builder, input_rows_count);
/// Store the result.
const ColumnWithTypeAndName & source_col = source_block.getByPosition(2);
@ -866,21 +867,21 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
}
void FunctionArrayElement::perform(Block & block, const ColumnNumbers & arguments, size_t result,
ArrayImpl::NullMapBuilder & builder)
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count)
{
if (executeTuple(block, arguments, result))
if (executeTuple(block, arguments, result, input_rows_count))
{
}
else if (!block.getByPosition(arguments[1]).column->isColumnConst())
{
if (!( executeArgument<UInt8>(block, arguments, result, builder)
|| executeArgument<UInt16>(block, arguments, result, builder)
|| executeArgument<UInt32>(block, arguments, result, builder)
|| executeArgument<UInt64>(block, arguments, result, builder)
|| executeArgument<Int8>(block, arguments, result, builder)
|| executeArgument<Int16>(block, arguments, result, builder)
|| executeArgument<Int32>(block, arguments, result, builder)
|| executeArgument<Int64>(block, arguments, result, builder)))
if (!(executeArgument<UInt8>(block, arguments, result, builder, input_rows_count)
|| executeArgument<UInt16>(block, arguments, result, builder, input_rows_count)
|| executeArgument<UInt32>(block, arguments, result, builder, input_rows_count)
|| executeArgument<UInt64>(block, arguments, result, builder, input_rows_count)
|| executeArgument<Int8>(block, arguments, result, builder, input_rows_count)
|| executeArgument<Int16>(block, arguments, result, builder, input_rows_count)
|| executeArgument<Int32>(block, arguments, result, builder, input_rows_count)
|| executeArgument<Int64>(block, arguments, result, builder, input_rows_count)))
throw Exception("Second argument for function " + getName() + " must must have UInt or Int type.",
ErrorCodes::ILLEGAL_COLUMN);
}
@ -889,7 +890,7 @@ void FunctionArrayElement::perform(Block & block, const ColumnNumbers & argument
Field index = (*block.getByPosition(arguments[1]).column)[0];
if (builder)
builder.initSink(block.rows());
builder.initSink(input_rows_count);
if (index == UInt64(0))
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
@ -934,7 +935,7 @@ DataTypePtr FunctionArrayEnumerate::getReturnTypeImpl(const DataTypes & argument
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>());
}
void FunctionArrayEnumerate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayEnumerate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
if (const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
{
@ -992,7 +993,7 @@ DataTypePtr FunctionArrayUniq::getReturnTypeImpl(const DataTypes & arguments) co
return std::make_shared<DataTypeUInt32>();
}
void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
Columns array_columns(arguments.size());
const ColumnArray::Offsets * offsets = nullptr;
@ -1301,7 +1302,7 @@ DataTypePtr FunctionArrayEnumerateUniq::getReturnTypeImpl(const DataTypes & argu
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>());
}
void FunctionArrayEnumerateUniq::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayEnumerateUniq::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
Columns array_columns(arguments.size());
const ColumnArray::Offsets * offsets = nullptr;
@ -1584,7 +1585,7 @@ namespace
{
namespace FunctionEmptyArrayToSingleImpl
{
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
if (const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
{
@ -1593,7 +1594,7 @@ namespace
auto nested_type = typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType();
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(
block.rows(),
input_rows_count,
Array{nested_type->getDefault()});
}
else
@ -1899,9 +1900,9 @@ namespace
}
}
void FunctionEmptyArrayToSingle::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionEmptyArrayToSingle::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
if (FunctionEmptyArrayToSingleImpl::executeConst(block, arguments, result))
if (FunctionEmptyArrayToSingleImpl::executeConst(block, arguments, result, input_rows_count))
return;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
@ -2017,7 +2018,7 @@ bool FunctionRange::executeInternal(Block & block, const IColumn * arg, const si
return false;
}
void FunctionRange::executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result)
void FunctionRange::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const auto col = block.getByPosition(arguments[0]).column.get();
@ -2054,9 +2055,9 @@ DataTypePtr FunctionArrayReverse::getReturnTypeImpl(const DataTypes & arguments)
return arguments[0];
}
void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
if (executeConst(block, arguments, result))
if (executeConst(block, arguments, result, input_rows_count))
return;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
@ -2111,7 +2112,8 @@ void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & argu
block.getByPosition(result).column = std::move(res_ptr);
}
bool FunctionArrayReverse::executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
bool FunctionArrayReverse::executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
size_t input_rows_count)
{
if (const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
{
@ -2123,7 +2125,7 @@ bool FunctionArrayReverse::executeConst(Block & block, const ColumnNumbers & arg
for (size_t i = 0; i < size; ++i)
res[i] = arr[size - i - 1];
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(block.rows(), res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, res);
return true;
}
@ -2399,7 +2401,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName
void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
IAggregateFunction & agg_func = *aggregate_function.get();
std::unique_ptr<char[]> place_holder { new char[agg_func.sizeOfData()] };
@ -2407,7 +2409,7 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum
std::unique_ptr<Arena> arena = agg_func.allocatesMemoryInArena() ? std::make_unique<Arena>() : nullptr;
size_t rows = block.rows();
size_t rows = input_rows_count;
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
std::vector<ColumnPtr> materialized_columns;
@ -2521,19 +2523,19 @@ DataTypePtr FunctionArrayConcat::getReturnTypeImpl(const DataTypes & arguments)
return getLeastSupertype(arguments);
}
void FunctionArrayConcat::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayConcat::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const DataTypePtr & return_type = block.getByPosition(result).type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
auto result_column = return_type->createColumn();
size_t rows = block.rows();
size_t rows = input_rows_count;
size_t num_args = arguments.size();
Columns preprocessed_columns(num_args);
@ -2614,13 +2616,13 @@ DataTypePtr FunctionArraySlice::getReturnTypeImpl(const DataTypes & arguments) c
return arguments[0];
}
void FunctionArraySlice::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArraySlice::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const auto & return_type = block.getByPosition(result).type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
@ -2716,13 +2718,13 @@ DataTypePtr FunctionArrayPush::getReturnTypeImpl(const DataTypes & arguments) co
return std::make_shared<DataTypeArray>(getLeastSupertype(types));
}
void FunctionArrayPush::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayPush::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const auto & return_type = block.getByPosition(result).type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
@ -2801,13 +2803,13 @@ DataTypePtr FunctionArrayPop::getReturnTypeImpl(const DataTypes & arguments) con
return arguments[0];
}
void FunctionArrayPop::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayPop::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const auto & return_type = block.getByPosition(result).type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
@ -2875,9 +2877,9 @@ DataTypePtr FunctionArrayHasAllAny::getReturnTypeImpl(const DataTypes & argument
return std::make_shared<DataTypeUInt8>();
}
void FunctionArrayHasAllAny::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayHasAllAny::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
size_t rows = block.rows();
size_t rows = input_rows_count;
size_t num_args = arguments.size();
auto result_column = ColumnUInt8::create(rows);
@ -3134,7 +3136,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(con
return arrays;
}
void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const auto & return_type = block.getByPosition(result).type;
auto return_type_array = checkAndGetDataType<DataTypeArray>(return_type.get());
@ -3146,7 +3148,7 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
if (typeid_cast<const DataTypeNothing *>(nested_return_type.get()))
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
@ -3352,13 +3354,13 @@ DataTypePtr FunctionArrayResize::getReturnTypeImpl(const DataTypes & arguments)
return std::make_shared<DataTypeArray>(getLeastSupertype({array_type->getNestedType(), arguments[2]}));
}
void FunctionArrayResize::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionArrayResize::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const auto & return_type = block.getByPosition(result).type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}

View File

@ -92,7 +92,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
String getName() const override;
@ -121,10 +121,11 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
void perform(Block & block, const ColumnNumbers & arguments, size_t result, ArrayImpl::NullMapBuilder & builder);
void perform(Block & block, const ColumnNumbers & arguments, size_t result,
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count);
template <typename DataType>
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
@ -149,15 +150,17 @@ private:
ArrayImpl::NullMapBuilder & builder);
template <typename IndexType>
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
ArrayImpl::NullMapBuilder & builder);
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
size_t input_rows_count);
template <typename IndexType>
bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result, ArrayImpl::NullMapBuilder & builder);
bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result,
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count);
/** For a tuple array, the function is evaluated component-wise for each element of the tuple.
*/
bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result);
bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count);
};
@ -1036,7 +1039,7 @@ public:
}
/// Perform function on the given block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
/// If one or both arguments passed to this function are nullable,
/// we create a new block that contains non-nullable arguments:
@ -1184,7 +1187,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
@ -1204,7 +1207,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
@ -1243,7 +1246,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
@ -1295,13 +1298,13 @@ private:
return std::make_shared<DataTypeArray>(std::make_shared<DataType>());
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
using UnderlyingColumnType = typename TypeToColumnType<typename DataType::FieldType>::ColumnType;
block.getByPosition(result).column = ColumnArray::create(
UnderlyingColumnType::create(),
ColumnArray::ColumnOffsets::create(block.rows(), 0));
ColumnArray::ColumnOffsets::create(input_rows_count, 0));
}
};
@ -1325,7 +1328,7 @@ private:
template <typename T>
bool executeInternal(Block & block, const IColumn * arg, const size_t result);
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
@ -1342,7 +1345,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
@ -1359,10 +1362,11 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result);
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
size_t input_rows_count);
template <typename T>
bool executeNumber(
@ -1404,7 +1408,7 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
/// lazy initialization in getReturnTypeImpl
/// TODO: init in FunctionBuilder
@ -1426,7 +1430,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
bool useDefaultImplementationForConstants() const override { return true; }
@ -1448,7 +1452,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForNulls() const override { return false; }
@ -1468,7 +1472,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForNulls() const override { return false; }
@ -1509,7 +1513,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForNulls() const override { return false; }
@ -1553,7 +1557,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
bool useDefaultImplementationForConstants() const override { return true; }
@ -1610,7 +1614,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
bool useDefaultImplementationForConstants() const override { return true; }
@ -1655,7 +1659,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForNulls() const override { return false; }

View File

@ -13,6 +13,8 @@
#include <ext/range.h>
#include <unicode/ucnv.h>
#include <string>
#include <memory>
namespace DB
@ -179,7 +181,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnWithTypeAndName & arg_from = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & arg_charset_from = block.getByPosition(arguments[1]);

View File

@ -86,7 +86,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto & col_type_name = block.getByPosition(arguments[0]);
const ColumnPtr & column = col_type_name.column;
@ -166,7 +166,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto & col_type_name = block.getByPosition(arguments[0]);
const ColumnPtr & column = col_type_name.column;
@ -414,7 +414,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
@ -528,7 +528,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
@ -611,7 +611,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
@ -664,7 +664,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto & col_type_name = block.getByPosition(arguments[0]);
const ColumnPtr & column = col_type_name.column;
@ -742,7 +742,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
@ -852,7 +852,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
@ -919,7 +919,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnWithTypeAndName & col_type_name = block.getByPosition(arguments[0]);
const ColumnPtr & column = col_type_name.column;
@ -1021,7 +1021,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnWithTypeAndName & col_type_name = block.getByPosition(arguments[0]);
const ColumnPtr & column = col_type_name.column;
@ -1113,12 +1113,12 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
auto col_res = ColumnVector<UInt128>::create();
typename ColumnVector<UInt128>::Container & vec_to = col_res->getData();
size_t size = block.rows();
size_t size = input_rows_count;
vec_to.resize(size);
Rand64Impl::execute(reinterpret_cast<UInt64 *>(&vec_to[0]), vec_to.size() * 2);
@ -1334,7 +1334,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & res_column = block.getByPosition(result).column;
@ -1397,7 +1397,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
@ -1507,7 +1507,7 @@ public:
}
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * in_column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & out_column = block.getByPosition(result).column;
@ -1645,7 +1645,7 @@ public:
}
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & res_column = block.getByPosition(result).column;

View File

@ -16,53 +16,61 @@ void registerFunctionsComparison(FunctionFactory & factory)
}
template <>
void FunctionComparison<EqualsOp, NameEquals>::executeTupleImpl(
Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size)
void FunctionComparison<EqualsOp, NameEquals>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleEqualityImpl<FunctionComparison<EqualsOp, NameEquals>, FunctionAnd>(block, result, x, y, tuple_size);
return executeTupleEqualityImpl<FunctionComparison<EqualsOp, NameEquals>, FunctionAnd>(block, result, x, y,
tuple_size, input_rows_count);
}
template <>
void FunctionComparison<NotEqualsOp, NameNotEquals>::executeTupleImpl(
Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size)
void FunctionComparison<NotEqualsOp, NameNotEquals>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleEqualityImpl<FunctionComparison<NotEqualsOp, NameNotEquals>, FunctionOr>(block, result, x, y, tuple_size);
return executeTupleEqualityImpl<FunctionComparison<NotEqualsOp, NameNotEquals>, FunctionOr>(block, result, x, y,
tuple_size, input_rows_count);
}
template <>
void FunctionComparison<LessOp, NameLess>::executeTupleImpl(
Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size)
void FunctionComparison<LessOp, NameLess>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleLessGreaterImpl<
FunctionComparison<LessOp, NameLess>,
FunctionComparison<LessOp, NameLess>>(block, result, x, y, tuple_size);
FunctionComparison<LessOp, NameLess>,
FunctionComparison<LessOp, NameLess>>(block, result, x, y, tuple_size, input_rows_count);
}
template <>
void FunctionComparison<GreaterOp, NameGreater>::executeTupleImpl(
Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size)
void FunctionComparison<GreaterOp, NameGreater>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleLessGreaterImpl<
FunctionComparison<GreaterOp, NameGreater>,
FunctionComparison<GreaterOp, NameGreater>>(block, result, x, y, tuple_size);
FunctionComparison<GreaterOp, NameGreater>,
FunctionComparison<GreaterOp, NameGreater>>(block, result, x, y, tuple_size, input_rows_count);
}
template <>
void FunctionComparison<LessOrEqualsOp, NameLessOrEquals>::executeTupleImpl(
Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size)
void FunctionComparison<LessOrEqualsOp, NameLessOrEquals>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleLessGreaterImpl<
FunctionComparison<LessOp, NameLess>,
FunctionComparison<LessOrEqualsOp, NameLessOrEquals>>(block, result, x, y, tuple_size);
FunctionComparison<LessOp, NameLess>,
FunctionComparison<LessOrEqualsOp, NameLessOrEquals>>(block, result, x, y, tuple_size, input_rows_count);
}
template <>
void FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>::executeTupleImpl(
Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size)
void FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>::executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count)
{
return executeTupleLessGreaterImpl<
FunctionComparison<GreaterOp, NameGreater>,
FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>>(block, result, x, y, tuple_size);
FunctionComparison<GreaterOp, NameGreater>,
FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>>(block, result, x, y, tuple_size, input_rows_count);
}
}

View File

@ -758,7 +758,7 @@ private:
void executeDateOrDateTimeOrEnumWithConstString(
Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped,
const DataTypePtr & left_type, const DataTypePtr & right_type, bool left_is_num)
const DataTypePtr & left_type, const DataTypePtr & right_type, bool left_is_num, size_t input_rows_count)
{
/// This is no longer very special case - comparing dates, datetimes, and enumerations with a string constant.
const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped;
@ -794,7 +794,7 @@ private:
if (!in.eof())
throw Exception("String is too long for Date: " + string_value.toString());
ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(block.rows(), UInt64(date));
ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(input_rows_count, UInt64(date));
const ColumnConst * parsed_const_date = static_cast<const ColumnConst *>(parsed_const_date_holder.get());
executeNumLeftType<DataTypeDate::FieldType>(block, result,
left_is_num ? col_left_untyped : parsed_const_date,
@ -808,7 +808,7 @@ private:
if (!in.eof())
throw Exception("String is too long for DateTime: " + string_value.toString());
ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(block.rows(), UInt64(date_time));
ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(input_rows_count, UInt64(date_time));
const ColumnConst * parsed_const_date_time = static_cast<const ColumnConst *>(parsed_const_date_time_holder.get());
executeNumLeftType<DataTypeDateTime::FieldType>(block, result,
left_is_num ? col_left_untyped : parsed_const_date_time,
@ -822,7 +822,7 @@ private:
if (!in.eof())
throw Exception("String is too long for UUID: " + string_value.toString());
ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(block.rows(), UInt128(uuid));
ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(input_rows_count, UInt128(uuid));
const ColumnConst * parsed_const_uuid = static_cast<const ColumnConst *>(parsed_const_uuid_holder.get());
executeNumLeftType<DataTypeUUID::FieldType>(block, result,
left_is_num ? col_left_untyped : parsed_const_uuid,
@ -831,29 +831,30 @@ private:
else if (is_enum8)
executeEnumWithConstString<DataTypeEnum8>(block, result, column_number, column_string,
number_type, left_is_num);
number_type, left_is_num, input_rows_count);
else if (is_enum16)
executeEnumWithConstString<DataTypeEnum16>(block, result, column_number, column_string,
number_type, left_is_num);
number_type, left_is_num, input_rows_count);
}
/// Comparison between DataTypeEnum<T> and string constant containing the name of an enum element
template <typename EnumType>
void executeEnumWithConstString(
Block & block, const size_t result, const IColumn * column_number, const ColumnConst * column_string,
const IDataType * type_untyped, const bool left_is_num)
const IDataType * type_untyped, const bool left_is_num, size_t input_rows_count)
{
const auto type = static_cast<const EnumType *>(type_untyped);
const Field x = nearestFieldType(type->getValue(column_string->getValue<String>()));
const auto enum_col = type->createColumnConst(block.rows(), x);
const auto enum_col = type->createColumnConst(input_rows_count, x);
executeNumLeftType<typename EnumType::FieldType>(block, result,
left_is_num ? column_number : enum_col.get(),
left_is_num ? enum_col.get() : column_number);
}
void executeTuple(Block & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1)
void executeTuple(Block & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1,
size_t input_rows_count)
{
/** We will lexicographically compare the tuples. This is done as follows:
* x == y : x1 == y1 && x2 == y2 ...
@ -902,13 +903,16 @@ private:
y[i].column = y_columns[i];
}
executeTupleImpl(block, result, x, y, tuple_size);
executeTupleImpl(block, result, x, y, tuple_size, input_rows_count);
}
void executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size);
void executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size,
size_t input_rows_count);
template <typename ComparisonFunction, typename ConvolutionFunction>
void executeTupleEqualityImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size)
void executeTupleEqualityImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y,
size_t tuple_size, size_t input_rows_count)
{
ComparisonFunction func_compare;
ConvolutionFunction func_convolution;
@ -921,7 +925,7 @@ private:
/// Comparison of the elements.
tmp_block.insert({ nullptr, std::make_shared<DataTypeUInt8>(), "" });
func_compare.execute(tmp_block, {i * 3, i * 3 + 1}, i * 3 + 2);
func_compare.execute(tmp_block, {i * 3, i * 3 + 1}, i * 3 + 2, input_rows_count);
}
/// Logical convolution.
@ -931,12 +935,13 @@ private:
for (size_t i = 0; i < tuple_size; ++i)
convolution_args[i] = i * 3 + 2;
func_convolution.execute(tmp_block, convolution_args, tuple_size * 3);
func_convolution.execute(tmp_block, convolution_args, tuple_size * 3, input_rows_count);
block.getByPosition(result).column = tmp_block.getByPosition(tuple_size * 3).column;
}
template <typename HeadComparisonFunction, typename TailComparisonFunction>
void executeTupleLessGreaterImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size)
void executeTupleLessGreaterImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x,
const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count)
{
HeadComparisonFunction func_compare_head;
TailComparisonFunction func_compare_tail;
@ -956,14 +961,14 @@ private:
if (i + 1 != tuple_size)
{
func_compare_head.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2);
func_compare_head.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2, input_rows_count);
tmp_block.insert({ nullptr, std::make_shared<DataTypeUInt8>(), "" });
func_equals.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 3);
func_equals.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 3, input_rows_count);
}
else
func_compare_tail.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2);
func_compare_tail.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2, input_rows_count);
}
/// Combination. Complex code - make a drawing. It can be replaced by a recursive comparison of tuples.
@ -971,9 +976,9 @@ private:
while (i > 0)
{
tmp_block.insert({ nullptr, std::make_shared<DataTypeUInt8>(), "" });
func_and.execute(tmp_block, { tmp_block.columns() - 2, (i - 1) * 4 + 3 }, tmp_block.columns() - 1);
func_and.execute(tmp_block, {tmp_block.columns() - 2, (i - 1) * 4 + 3}, tmp_block.columns() - 1, input_rows_count);
tmp_block.insert({ nullptr, std::make_shared<DataTypeUInt8>(), "" });
func_or.execute(tmp_block, { tmp_block.columns() - 2, (i - 1) * 4 + 2 }, tmp_block.columns() - 1);
func_or.execute(tmp_block, {tmp_block.columns() - 2, (i - 1) * 4 + 2}, tmp_block.columns() - 1, input_rows_count);
--i;
}
@ -1094,7 +1099,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto & col_with_type_and_name_left = block.getByPosition(arguments[0]);
const auto & col_with_type_and_name_right = block.getByPosition(arguments[1]);
@ -1122,7 +1127,7 @@ public:
ErrorCodes::ILLEGAL_COLUMN);
}
else if (checkAndGetDataType<DataTypeTuple>(col_with_type_and_name_left.type.get()))
executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right);
executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count);
else if (!left_is_num && !right_is_num && executeString(block, result, col_left_untyped, col_right_untyped))
;
else if (col_with_type_and_name_left.type->equals(*col_with_type_and_name_right.type))
@ -1131,7 +1136,7 @@ public:
executeDateOrDateTimeOrEnumWithConstString(
block, result, col_left_untyped, col_right_untyped,
col_with_type_and_name_left.type, col_with_type_and_name_right.type,
left_is_num);
left_is_num, input_rows_count);
}
};

View File

@ -4,6 +4,7 @@
#include <Functions/FunctionFactory.h>
#include <Columns/ColumnNullable.h>
#include <Interpreters/castColumn.h>
#include <vector>
namespace DB
@ -40,7 +41,7 @@ String FunctionMultiIf::getName() const
}
void FunctionMultiIf::executeImpl(Block & block, const ColumnNumbers & args, size_t result)
void FunctionMultiIf::executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
/** We will gather values from columns in branches to result column,
* depending on values of conditions.
@ -123,7 +124,7 @@ void FunctionMultiIf::executeImpl(Block & block, const ColumnNumbers & args, siz
break;
}
size_t rows = block.rows();
size_t rows = input_rows_count;
MutableColumnPtr res = return_type->createColumn();
for (size_t i = 0; i < rows; ++i)
@ -274,7 +275,7 @@ DataTypePtr FunctionCaseWithExpression::getReturnTypeImpl(const DataTypes & args
return fun_transform.getReturnType(transform_args);
}
void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers & args, size_t result)
void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
if (!args.size())
throw Exception{"Function " + getName() + " expects at least 1 arguments",
@ -322,14 +323,14 @@ void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers
size_t dst_array_pos = temp_block.columns();
temp_block.insert({nullptr, dst_array_type, ""});
fun_array.execute(temp_block, src_array_args, src_array_pos);
fun_array.execute(temp_block, dst_array_args, dst_array_pos);
fun_array.execute(temp_block, src_array_args, src_array_pos, input_rows_count);
fun_array.execute(temp_block, dst_array_args, dst_array_pos, input_rows_count);
/// Execute transform.
FunctionTransform fun_transform;
ColumnNumbers transform_args{args.front(), src_array_pos, dst_array_pos, args.back()};
fun_transform.execute(temp_block, transform_args, result);
fun_transform.execute(temp_block, transform_args, result, input_rows_count);
/// Put the result into the original block.
block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column);

View File

@ -174,7 +174,8 @@ private:
[[maybe_unused]] Block & block,
[[maybe_unused]] const ColumnNumbers & arguments,
[[maybe_unused]] size_t result,
[[maybe_unused]] const ColumnArray * col_left_array)
[[maybe_unused]] const ColumnArray * col_left_array,
[[maybe_unused]] size_t input_rows_count)
{
if constexpr (std::is_same_v<NumberTraits::Error, typename NumberTraits::ResultOfIf<T0, T1>::Type>)
return false;
@ -202,7 +203,7 @@ private:
conditional(
NumericArraySource<T0>(*col_left_array),
NumericArraySource<T1>(*col_right_array),
NumericArraySink<ResultType>(static_cast<ColumnArray &>(*res), block.rows()),
NumericArraySink<ResultType>(static_cast<ColumnArray &>(*res), input_rows_count),
cond_col->getData());
block.getByPosition(result).column = std::move(res);
@ -218,7 +219,7 @@ private:
conditional(
NumericArraySource<T0>(*col_left_array),
ConstSource<NumericArraySource<T1>>(*col_right_const_array),
NumericArraySink<ResultType>(static_cast<ColumnArray &>(*res), block.rows()),
NumericArraySink<ResultType>(static_cast<ColumnArray &>(*res), input_rows_count),
cond_col->getData());
block.getByPosition(result).column = std::move(res);
@ -234,7 +235,8 @@ private:
[[maybe_unused]] Block & block,
[[maybe_unused]] const ColumnNumbers & arguments,
[[maybe_unused]] size_t result,
[[maybe_unused]] const ColumnConst * col_left_const_array)
[[maybe_unused]] const ColumnConst * col_left_const_array,
[[maybe_unused]] size_t input_rows_count)
{
if constexpr (std::is_same_v<NumberTraits::Error, typename NumberTraits::ResultOfIf<T0, T1>::Type>)
return false;
@ -262,7 +264,7 @@ private:
conditional(
ConstSource<NumericArraySource<T0>>(*col_left_const_array),
NumericArraySource<T1>(*col_right_array),
NumericArraySink<ResultType>(static_cast<ColumnArray &>(*res), block.rows()),
NumericArraySink<ResultType>(static_cast<ColumnArray &>(*res), input_rows_count),
cond_col->getData());
block.getByPosition(result).column = std::move(res);
@ -278,7 +280,7 @@ private:
conditional(
ConstSource<NumericArraySource<T0>>(*col_left_const_array),
ConstSource<NumericArraySource<T1>>(*col_right_const_array),
NumericArraySink<ResultType>(static_cast<ColumnArray &>(*res), block.rows()),
NumericArraySink<ResultType>(static_cast<ColumnArray &>(*res), input_rows_count),
cond_col->getData());
block.getByPosition(result).column = std::move(res);
@ -289,7 +291,7 @@ private:
}
template <typename T0>
bool executeLeftType(const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
bool executeLeftType(const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const IColumn * col_left_untyped = block.getByPosition(arguments[1]).column.get();
@ -352,16 +354,16 @@ private:
}
else if (col_arr_left && col_arr_left_elems)
{
if ( executeRightTypeArray<T0, UInt8>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, UInt16>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, UInt32>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, UInt64>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, Int8>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, Int16>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, Int32>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, Int64>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, Float32>(cond_col, block, arguments, result, col_arr_left)
|| executeRightTypeArray<T0, Float64>(cond_col, block, arguments, result, col_arr_left))
if ( executeRightTypeArray<T0, UInt8>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, UInt16>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, UInt32>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, UInt64>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, Int8>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, Int16>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, Int32>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, Int64>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, Float32>(cond_col, block, arguments, result, col_arr_left, input_rows_count)
|| executeRightTypeArray<T0, Float64>(cond_col, block, arguments, result, col_arr_left, input_rows_count))
return true;
else
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
@ -370,16 +372,16 @@ private:
}
else if (col_const_arr_left && checkColumn<ColumnVector<T0>>(&static_cast<const ColumnArray &>(col_const_arr_left->getDataColumn()).getData()))
{
if ( executeConstRightTypeArray<T0, UInt8>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, UInt16>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, UInt32>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, UInt64>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, Int8>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, Int16>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, Int32>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, Int64>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, Float32>(cond_col, block, arguments, result, col_const_arr_left)
|| executeConstRightTypeArray<T0, Float64>(cond_col, block, arguments, result, col_const_arr_left))
if ( executeConstRightTypeArray<T0, UInt8>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, UInt16>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, UInt32>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, UInt64>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, Int8>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, Int16>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, Int32>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, Int64>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, Float32>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count)
|| executeConstRightTypeArray<T0, Float64>(cond_col, block, arguments, result, col_const_arr_left, input_rows_count))
return true;
else
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
@ -509,7 +511,7 @@ private:
return false;
}
bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result)
bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
/// Calculate function for each corresponding elements of tuples.
@ -552,7 +554,7 @@ private:
temporary_block.insert({col2_contents[i], type2.getElements()[i], {}});
/// temporary_block will be: cond, res_0, ..., res_i, then_i, else_i
executeImpl(temporary_block, {0, i + 2, i + 3}, i + 1);
executeImpl(temporary_block, {0, i + 2, i + 3}, i + 1, input_rows_count);
temporary_block.erase(i + 3);
temporary_block.erase(i + 2);
@ -565,7 +567,7 @@ private:
return true;
}
bool executeForNullableCondition(Block & block, const ColumnNumbers & arguments, size_t result)
bool executeForNullableCondition(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const ColumnWithTypeAndName & arg_cond = block.getByPosition(arguments[0]);
bool cond_is_null = arg_cond.column->onlyNull();
@ -573,7 +575,7 @@ private:
if (cond_is_null)
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count);
return true;
}
@ -587,7 +589,7 @@ private:
block.getByPosition(result)
};
executeImpl(temporary_block, {0, 1, 2}, 3);
executeImpl(temporary_block, {0, 1, 2}, 3, temporary_block.rows());
const ColumnPtr & result_column = temporary_block.getByPosition(3).column;
if (result_column->isColumnNullable())
@ -599,7 +601,7 @@ private:
}
else if (result_column->onlyNull())
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count);
return true;
}
else
@ -637,7 +639,7 @@ private:
return column;
}
bool executeForNullableThenElse(Block & block, const ColumnNumbers & arguments, size_t result)
bool executeForNullableThenElse(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const ColumnWithTypeAndName & arg_cond = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & arg_then = block.getByPosition(arguments[1]);
@ -660,14 +662,14 @@ private:
{
then_is_nullable
? static_cast<const ColumnNullable *>(arg_then.column.get())->getNullMapColumnPtr()
: DataTypeUInt8().createColumnConstWithDefaultValue(block.rows()),
: DataTypeUInt8().createColumnConstWithDefaultValue(input_rows_count),
std::make_shared<DataTypeUInt8>(),
""
},
{
else_is_nullable
? static_cast<const ColumnNullable *>(arg_else.column.get())->getNullMapColumnPtr()
: DataTypeUInt8().createColumnConstWithDefaultValue(block.rows()),
: DataTypeUInt8().createColumnConstWithDefaultValue(input_rows_count),
std::make_shared<DataTypeUInt8>(),
""
},
@ -678,7 +680,7 @@ private:
}
});
executeImpl(temporary_block, {0, 1, 2}, 3);
executeImpl(temporary_block, {0, 1, 2}, 3, temporary_block.rows());
result_null_mask = temporary_block.getByPosition(3).column;
}
@ -706,7 +708,7 @@ private:
}
});
executeImpl(temporary_block, {0, 1, 2}, 3);
executeImpl(temporary_block, {0, 1, 2}, 3, temporary_block.rows());
result_nested_column = temporary_block.getByPosition(3).column;
}
@ -716,7 +718,7 @@ private:
return true;
}
bool executeForNullThenElse(Block & block, const ColumnNumbers & arguments, size_t result)
bool executeForNullThenElse(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const ColumnWithTypeAndName & arg_cond = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & arg_then = block.getByPosition(arguments[1]);
@ -730,7 +732,7 @@ private:
if (then_is_null && else_is_null)
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count);
return true;
}
@ -758,7 +760,7 @@ private:
else if (cond_const_col)
{
if (cond_const_col->getValue<UInt8>())
block.getByPosition(result).column = block.getByPosition(result).type->createColumn()->cloneResized(block.rows());
block.getByPosition(result).column = block.getByPosition(result).type->createColumn()->cloneResized(input_rows_count);
else
block.getByPosition(result).column = makeNullableColumnIfNot(arg_else.column);
}
@ -774,7 +776,7 @@ private:
{
if (cond_col)
{
size_t size = block.rows();
size_t size = input_rows_count;
auto & null_map_data = cond_col->getData();
auto negated_null_map = ColumnUInt8::create();
@ -802,7 +804,7 @@ private:
if (cond_const_col->getValue<UInt8>())
block.getByPosition(result).column = makeNullableColumnIfNot(arg_then.column);
else
block.getByPosition(result).column = block.getByPosition(result).type->createColumn()->cloneResized(block.rows());
block.getByPosition(result).column = block.getByPosition(result).type->createColumn()->cloneResized(input_rows_count);
}
else
throw Exception("Illegal column " + arg_cond.column->getName() + " of first argument of function " + getName()
@ -841,11 +843,11 @@ public:
return getLeastSupertype({arguments[1], arguments[2]});
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
if (executeForNullableCondition(block, arguments, result)
|| executeForNullThenElse(block, arguments, result)
|| executeForNullableThenElse(block, arguments, result))
if (executeForNullableCondition(block, arguments, result, input_rows_count)
|| executeForNullThenElse(block, arguments, result, input_rows_count)
|| executeForNullableThenElse(block, arguments, result, input_rows_count))
return;
const ColumnWithTypeAndName & arg_cond = block.getByPosition(arguments[0]);
@ -874,19 +876,19 @@ public:
if (cond_col)
{
if (!( executeLeftType<UInt8>(cond_col, block, arguments, result)
|| executeLeftType<UInt16>(cond_col, block, arguments, result)
|| executeLeftType<UInt32>(cond_col, block, arguments, result)
|| executeLeftType<UInt64>(cond_col, block, arguments, result)
|| executeLeftType<Int8>(cond_col, block, arguments, result)
|| executeLeftType<Int16>(cond_col, block, arguments, result)
|| executeLeftType<Int32>(cond_col, block, arguments, result)
|| executeLeftType<Int64>(cond_col, block, arguments, result)
|| executeLeftType<Float32>(cond_col, block, arguments, result)
|| executeLeftType<Float64>(cond_col, block, arguments, result)
if (!( executeLeftType<UInt8>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<UInt16>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<UInt32>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<UInt64>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<Int8>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<Int16>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<Int32>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<Int64>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<Float32>(cond_col, block, arguments, result, input_rows_count)
|| executeLeftType<Float64>(cond_col, block, arguments, result, input_rows_count)
|| executeString(cond_col, block, arguments, result)
|| executeGenericArray(cond_col, block, arguments, result)
|| executeTuple(block, arguments, result)))
|| executeTuple(block, arguments, result, input_rows_count)))
throw Exception("Illegal columns " + arg_then.column->getName()
+ " and " + arg_else.column->getName()
+ " of second (then) and third (else) arguments of function " + getName(),
@ -928,7 +930,7 @@ public:
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForNulls() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override;
private:
const Context & context;
@ -949,7 +951,7 @@ public:
size_t getNumberOfArguments() const override { return 0; }
String getName() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override;
private:
const Context & context;

View File

@ -115,7 +115,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (block.getByPosition(arguments[1]).column->isColumnConst())
executeConstBuckets(block, arguments, result);

View File

@ -79,7 +79,7 @@ struct ConvertImpl
using FromFieldType = typename FromDataType::FieldType;
using ToFieldType = typename ToDataType::FieldType;
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
if (const ColumnVector<FromFieldType> * col_from
= checkAndGetColumn<ColumnVector<FromFieldType>>(block.getByPosition(arguments[0]).column.get()))
@ -202,7 +202,7 @@ struct FormatImpl<DataTypeEnum<FieldType>>
template <typename FieldType, typename Name>
struct ConvertImpl<DataTypeEnum<FieldType>, DataTypeNumber<FieldType>, Name>
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
}
@ -214,7 +214,7 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
{
using FromFieldType = typename FromDataType::FieldType;
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const auto & col_with_type_and_name = block.getByPosition(arguments[0]);
const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);
@ -381,7 +381,7 @@ struct ConvertThroughParsing
return false;
}
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr;
const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr;
@ -409,7 +409,7 @@ struct ConvertThroughParsing
+ " of first argument of function " + Name::name,
ErrorCodes::ILLEGAL_COLUMN);
size_t size = block.rows();
size_t size = input_rows_count;
auto col_to = ColumnVector<ToFieldType>::create(size);
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
@ -560,7 +560,7 @@ struct ConvertImpl<DataTypeString, DataTypeUInt32, NameToUnixTimestamp>
template <typename T, typename Name>
struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name>
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
}
@ -573,7 +573,7 @@ struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name>
template <typename Name>
struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(block.getByPosition(arguments[0]).column.get()))
{
@ -702,11 +702,11 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
try
{
executeInternal(block, arguments, result);
executeInternal(block, arguments, result, input_rows_count);
}
catch (Exception & e)
{
@ -747,7 +747,7 @@ public:
}
private:
void executeInternal(Block & block, const ColumnNumbers & arguments, size_t result)
void executeInternal(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
if (!arguments.size())
throw Exception{"Function " + getName() + " expects at least 1 arguments",
@ -755,23 +755,25 @@ private:
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
if (checkDataType<DataTypeUInt8>(from_type)) ConvertImpl<DataTypeUInt8, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeUInt16>(from_type)) ConvertImpl<DataTypeUInt16, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeUInt32>(from_type)) ConvertImpl<DataTypeUInt32, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeUInt64>(from_type)) ConvertImpl<DataTypeUInt64, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeInt8>(from_type)) ConvertImpl<DataTypeInt8, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeInt16>(from_type)) ConvertImpl<DataTypeInt16, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeInt32>(from_type)) ConvertImpl<DataTypeInt32, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeInt64>(from_type)) ConvertImpl<DataTypeInt64, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeFloat32>(from_type)) ConvertImpl<DataTypeFloat32, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeFloat64>(from_type)) ConvertImpl<DataTypeFloat64, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeDate>(from_type)) ConvertImpl<DataTypeDate, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeDateTime>(from_type)) ConvertImpl<DataTypeDateTime, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeUUID>(from_type)) ConvertImpl<DataTypeUUID, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeString>(from_type)) ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeFixedString>(from_type)) ConvertImpl<DataTypeFixedString, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeEnum8>(from_type)) ConvertImpl<DataTypeEnum8, ToDataType, Name>::execute(block, arguments, result);
else if (checkDataType<DataTypeEnum16>(from_type)) ConvertImpl<DataTypeEnum16, ToDataType, Name>::execute(block, arguments, result);
if (checkDataType<DataTypeUInt8>(from_type)) ConvertImpl<DataTypeUInt8, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeUInt16>(from_type)) ConvertImpl<DataTypeUInt16, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeUInt32>(from_type)) ConvertImpl<DataTypeUInt32, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeUInt64>(from_type)) ConvertImpl<DataTypeUInt64, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeInt8>(from_type)) ConvertImpl<DataTypeInt8, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeInt16>(from_type)) ConvertImpl<DataTypeInt16, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeInt32>(from_type)) ConvertImpl<DataTypeInt32, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeInt64>(from_type)) ConvertImpl<DataTypeInt64, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeFloat32>(from_type)) ConvertImpl<DataTypeFloat32, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeFloat64>(from_type)) ConvertImpl<DataTypeFloat64, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeDate>(from_type)) ConvertImpl<DataTypeDate, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeDateTime>(from_type)) ConvertImpl<DataTypeDateTime, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeUUID>(from_type)) ConvertImpl<DataTypeUUID, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeString>(from_type))
ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeFixedString>(from_type))
ConvertImpl<DataTypeFixedString, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeEnum8>(from_type)) ConvertImpl<DataTypeEnum8, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeEnum16>(from_type)) ConvertImpl<DataTypeEnum16, ToDataType, Name>::execute(block, arguments, result, input_rows_count);
else
{
/// Generic conversion of any type to String.
@ -851,14 +853,19 @@ public:
return res;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
if (checkAndGetDataType<DataTypeString>(from_type))
ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(block, arguments, result);
ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(block,
arguments,
result, input_rows_count);
else if (checkAndGetDataType<DataTypeFixedString>(from_type))
ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(block, arguments, result);
ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(block,
arguments,
result,
input_rows_count);
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName()
+ ". Only String or FixedString argument is accepted for try-conversion function. For other arguments, use function without 'orZero' or 'orNull'.",
@ -899,13 +906,13 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto n = block.getByPosition(arguments[1]).column->getUInt(0);
return execute(block, arguments, result, n);
return executeForN(block, arguments, result, n);
}
static void execute(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
static void executeForN(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
{
const auto & column = block.getByPosition(arguments[0]).column;
@ -1188,7 +1195,7 @@ using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString<
class PreparedFunctionCast : public PreparedFunctionImpl
{
public:
using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t)>;
using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t, size_t)>;
explicit PreparedFunctionCast(WrapperType && wrapper_function, const char * name)
: wrapper_function(std::move(wrapper_function)), name(name) {}
@ -1196,14 +1203,14 @@ public:
String getName() const override { return name; }
protected:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
/// drop second argument, pass others
ColumnNumbers new_arguments{arguments.front()};
if (arguments.size() > 2)
new_arguments.insert(std::end(new_arguments), std::next(std::begin(arguments), 2), std::end(arguments));
wrapper_function(block, new_arguments, result);
wrapper_function(block, new_arguments, result, input_rows_count);
}
bool useDefaultImplementationForNulls() const override { return false; }
@ -1218,7 +1225,7 @@ private:
class FunctionCast final : public IFunctionBase
{
public:
using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t)>;
using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t, size_t)>;
using MonotonicityForRange = std::function<Monotonicity(const IDataType &, const Field &, const Field &)>;
FunctionCast(const Context & context, const char * name, MonotonicityForRange && monotonicity_for_range
@ -1269,9 +1276,9 @@ private:
function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" }));
}
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
function->execute(block, arguments, result);
function->execute(block, arguments, result, input_rows_count);
};
}
@ -1280,9 +1287,9 @@ private:
if (!from_type->isStringOrFixedString())
throw Exception{"CAST AS FixedString is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED};
return [N] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [N] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
FunctionToFixedString::execute(block, arguments, result, N);
FunctionToFixedString::executeForN(block, arguments, result, N);
};
}
@ -1291,7 +1298,7 @@ private:
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
{
return [] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
ConvertImplGenericFromString::execute(block, arguments, result);
};
@ -1319,7 +1326,7 @@ private:
const auto nested_function = prepare(from_nested_type, to_nested_type);
return [nested_function, from_nested_type, to_nested_type](
Block & block, const ColumnNumbers & arguments, const size_t result)
Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
const auto & array_arg = block.getByPosition(arguments.front());
@ -1333,7 +1340,7 @@ private:
};
/// convert nested column
nested_function(nested_block, {0}, 1);
nested_function(nested_block, {0}, 1, nested_block.rows());
/// set converted nested column to result
block.getByPosition(result).column = ColumnArray::create(nested_block.getByPosition(1).column, col_array->getOffsetsPtr());
@ -1348,7 +1355,7 @@ private:
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
{
return [] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
ConvertImplGenericFromString::execute(block, arguments, result);
};
@ -1373,7 +1380,7 @@ private:
element_wrappers.push_back(prepare(idx_type.second, to_element_types[idx_type.first]));
return [element_wrappers, from_element_types, to_element_types]
(Block & block, const ColumnNumbers & arguments, const size_t result)
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
const auto col = block.getByPosition(arguments.front()).column.get();
@ -1397,7 +1404,7 @@ private:
/// invoke conversion for each element
for (const auto & idx_element_wrapper : ext::enumerate(element_wrappers))
idx_element_wrapper.second(element_block, { idx_element_wrapper.first },
tuple_size + idx_element_wrapper.first);
tuple_size + idx_element_wrapper.first, input_rows_count);
Columns converted_columns(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
@ -1431,9 +1438,9 @@ private:
function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" }));
}
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
function->execute(block, arguments, result);
function->execute(block, arguments, result, input_rows_count);
};
}
else
@ -1470,7 +1477,7 @@ private:
WrapperType createStringToEnumWrapper() const
{
const char * function_name = name;
return [function_name] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [function_name] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
const auto first_col = block.getByPosition(arguments.front()).column.get();
@ -1498,7 +1505,7 @@ private:
WrapperType createIdentityWrapper(const DataTypePtr &) const
{
return [] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
block.getByPosition(result).column = block.getByPosition(arguments.front()).column;
};
@ -1507,10 +1514,10 @@ private:
WrapperType createNothingWrapper(const IDataType * to_type) const
{
ColumnPtr res = to_type->createColumnConstWithDefaultValue(1);
return [res] (Block & block, const ColumnNumbers &, const size_t result)
return [res] (Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
{
/// Column of Nothing type is trivially convertible to any other column
block.getByPosition(result).column = res->cloneResized(block.rows())->convertToFullColumnIfConst();
block.getByPosition(result).column = res->cloneResized(input_rows_count)->convertToFullColumnIfConst();
};
}
@ -1534,10 +1541,10 @@ private:
if (!nullable_conversion.result_is_nullable)
throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};
return [](Block & block, const ColumnNumbers &, const size_t result)
return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
{
auto & res = block.getByPosition(result);
res.column = res.type->createColumnConstWithDefaultValue(block.rows())->convertToFullColumnIfConst();
res.column = res.type->createColumnConstWithDefaultValue(input_rows_count)->convertToFullColumnIfConst();
};
}
@ -1548,7 +1555,7 @@ private:
if (nullable_conversion.result_is_nullable)
{
return [wrapper, nullable_conversion] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [wrapper, nullable_conversion] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
/// Create a temporary block on which to perform the operation.
auto & res = block.getByPosition(result);
@ -1566,7 +1573,7 @@ private:
tmp_block.insert({nullptr, nested_type, ""});
/// Perform the requested conversion.
wrapper(tmp_block, arguments, tmp_res_index);
wrapper(tmp_block, arguments, tmp_res_index, input_rows_count);
/// Wrap the result into a nullable column.
ColumnPtr null_map;
@ -1583,7 +1590,7 @@ private:
{
/// This is a conversion from an ordinary type to a nullable type.
/// So we create a trivial null map.
null_map = ColumnUInt8::create(block.rows(), 0);
null_map = ColumnUInt8::create(input_rows_count, 0);
}
const auto & tmp_res = tmp_block.getByPosition(tmp_res_index);
@ -1594,7 +1601,7 @@ private:
{
/// Conversion from Nullable to non-Nullable.
return [wrapper] (Block & block, const ColumnNumbers & arguments, const size_t result)
return [wrapper] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
Block tmp_block = createBlockWithNestedColumns(block, arguments, result);
@ -1608,7 +1615,7 @@ private:
throw Exception{"Cannot convert NULL value to non-Nullable type",
ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};
wrapper(tmp_block, arguments, result);
wrapper(tmp_block, arguments, result, input_rows_count);
block.getByPosition(result).column = tmp_block.getByPosition(result).column;
};
}

View File

@ -595,7 +595,7 @@ struct Transformer
template <typename FromType, typename ToType, typename Transform>
struct DateTimeTransformImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
using Op = Transformer<FromType, ToType, Transform>;
@ -668,14 +668,14 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
if (checkDataType<DataTypeDate>(from_type))
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result);
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
else if (checkDataType<DataTypeDateTime>(from_type))
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result);
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -989,7 +989,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
@ -1056,7 +1056,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 3}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
auto * unit_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!unit_column)
@ -1067,7 +1067,7 @@ public:
const IColumn & x = *block.getByPosition(arguments[1]).column;
const IColumn & y = *block.getByPosition(arguments[2]).column;
size_t rows = block.rows();
size_t rows = input_rows_count;
auto res = ColumnInt64::create(rows);
const DateLUTImpl & timezone_x = extractTimeZoneFromFunctionArguments(block, arguments, 3, 1);
@ -1210,10 +1210,10 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeUInt32().createColumnConst(
block.rows(),
input_rows_count,
static_cast<UInt64>(time(nullptr)));
}
};
@ -1239,10 +1239,10 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeUInt16().createColumnConst(
block.rows(),
input_rows_count,
UInt64(DateLUT::instance().toDayNum(time(nullptr))));
}
};
@ -1268,10 +1268,10 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeUInt16().createColumnConst(
block.rows(),
input_rows_count,
UInt64(DateLUT::instance().toDayNum(time(nullptr)) - 1));
}
};
@ -1307,7 +1307,7 @@ public:
return std::make_shared<DataTypeDateTime>(time_zone_name);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
}
@ -1338,7 +1338,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (const ColumnUInt32 * times = typeid_cast<const ColumnUInt32 *>(block.getByPosition(arguments[0]).column.get()))
{
@ -1467,7 +1467,7 @@ public:
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>());
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
auto starts = checkAndGetColumn<ColumnUInt32>(block.getByPosition(arguments[0]).column.get());
auto const_starts = checkAndGetColumnConst<ColumnUInt32>(block.getByPosition(arguments[0]).column.get());
@ -1497,7 +1497,7 @@ public:
{
Array const_res;
TimeSlotsImpl<UInt32>::constant_constant(const_starts->getValue<UInt32>(), const_durations->getValue<UInt32>(), const_res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(block.rows(), const_res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, const_res);
}
else
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()

View File

@ -220,7 +220,7 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
/// The dictionary key that defines the "point of view".
std::string dict_key;
@ -316,7 +316,7 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
/// The dictionary key that defines the "point of view".
std::string dict_key;
@ -452,7 +452,7 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
/// The dictionary key that defines the "point of view".
std::string dict_key;
@ -728,7 +728,7 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
RegionsNames::Language language = RegionsNames::Language::RU;

View File

@ -95,7 +95,7 @@ private:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
@ -108,7 +108,7 @@ private:
* This feature is controversial and implemented specially
* for backward compatibility with the case in Yandex Banner System.
*/
if (block.rows() == 0)
if (input_rows_count== 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
@ -260,13 +260,13 @@ private:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
@ -490,13 +490,13 @@ private:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
@ -756,13 +756,13 @@ private:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
@ -1031,13 +1031,13 @@ private:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
@ -1264,13 +1264,13 @@ private:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
@ -1424,13 +1424,13 @@ private:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();

View File

@ -8,6 +8,8 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnString.h>
#include <ext/range.h>
#include <string>
#include <memory>
namespace DB
{
@ -37,7 +39,7 @@ DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const DataTypes & arguments
return std::make_shared<DataTypeFloat64>();
}
void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const auto name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!name_col)

View File

@ -29,7 +29,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
const ExternalModels & models;

View File

@ -128,7 +128,7 @@ public:
return std::make_shared<DataTypeUInt64>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto in_untyped = block.getByPosition(arguments[0]).column.get();
const auto centroids_array_untyped = block.getByPosition(arguments[1]).column.get();

View File

@ -54,7 +54,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)
@ -147,7 +147,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)

View File

@ -14,6 +14,8 @@
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
#include <Common/ProfileEvents.h>
#include <string>
#include <memory>
namespace ProfileEvents
@ -142,7 +144,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * point_col = block.getByPosition(arguments[0]).column.get();

View File

@ -117,9 +117,9 @@ private:
}
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto size = block.rows();
const auto size = input_rows_count;
bool result_is_const{};
auto instrs = getInstructions(block, arguments, result_is_const);
@ -224,9 +224,9 @@ private:
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto size = block.rows();
const auto size = input_rows_count;
/// Prepare array of ellipses.
size_t ellipses_count = (arguments.size() - 2) / 4;

View File

@ -201,7 +201,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
{
@ -253,7 +253,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
{
@ -333,7 +333,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
@ -587,9 +587,9 @@ public:
return std::make_shared<DataTypeUInt64>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
size_t rows = block.rows();
size_t rows = input_rows_count;
auto col_to = ColumnUInt64::create(rows);
ColumnUInt64::Container & vec_to = col_to->getData();
@ -728,7 +728,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto arg_count = arguments.size();

View File

@ -855,7 +855,7 @@ public:
}
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (arguments.size() == 1)
{

View File

@ -293,7 +293,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
size_t num_arguments = arguments.size();
ColumnRawPtrs in(num_arguments);
@ -414,7 +414,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)

View File

@ -56,9 +56,9 @@ private:
return std::make_shared<DataTypeFloat64>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(block.rows(), Impl::value);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, Impl::value);
}
};
@ -125,7 +125,7 @@ private:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto arg = block.getByPosition(arguments[0]).column.get();
@ -387,7 +387,7 @@ private:
return false;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto left_arg = block.getByPosition(arguments[0]).column.get();

View File

@ -110,9 +110,9 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeString().createColumnConst(block.rows(), db_name);
block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, db_name);
}
};
@ -152,10 +152,10 @@ public:
/** convertToFullColumn needed because in distributed query processing,
* each server returns its own value.
*/
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(
block.rows(), Poco::Net::DNS::hostName())->convertToFullColumnIfConst();
input_rows_count, Poco::Net::DNS::hostName())->convertToFullColumnIfConst();
}
};
@ -190,7 +190,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
/// Execute the function on the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override;
};
@ -222,10 +222,10 @@ public:
}
/// Execute the function on the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column
= DataTypeString().createColumnConst(block.rows(), block.getByPosition(arguments[0]).type->getName());
= DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).type->getName());
}
};
@ -262,12 +262,12 @@ public:
throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
if (auto type = checkAndGetDataType<DataTypeEnum8>(block.getByPosition(arguments[0]).type.get()))
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(block.rows(), UInt64(type->getValues().size()));
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(type->getValues().size()));
else if (auto type = checkAndGetDataType<DataTypeEnum16>(block.getByPosition(arguments[0]).type.get()))
block.getByPosition(result).column = DataTypeUInt16().createColumnConst(block.rows(), UInt64(type->getValues().size()));
block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, UInt64(type->getValues().size()));
else
throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -302,10 +302,10 @@ public:
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column
= DataTypeString().createColumnConst(block.rows(), block.getByPosition(arguments[0]).column->getName());
= DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).column->getName());
}
};
@ -338,14 +338,14 @@ public:
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto & elem = block.getByPosition(arguments[0]);
/// Note that the result is not a constant, because it contains block size.
block.getByPosition(result).column
= DataTypeString().createColumnConst(block.rows(),
= DataTypeString().createColumnConst(input_rows_count,
elem.type->getName() + ", " + elem.column->dumpStructure())->convertToFullColumnIfConst();
}
};
@ -378,10 +378,10 @@ public:
return arguments[0];
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const IDataType & type = *block.getByPosition(arguments[0]).type;
block.getByPosition(result).column = type.createColumnConst(block.rows(), type.getDefault());
block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault());
}
};
@ -418,10 +418,9 @@ public:
return std::make_shared<DataTypeUInt64>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
size_t size = block.rows();
block.getByPosition(result).column = ColumnUInt64::create(size, size);
block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, input_rows_count);
}
};
@ -458,13 +457,12 @@ public:
return std::make_shared<DataTypeUInt64>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
size_t size = block.rows();
auto column = ColumnUInt64::create();
auto & data = column->getData();
data.resize(size);
for (size_t i = 0; i < size; ++i)
data.resize(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i)
data[i] = i;
block.getByPosition(result).column = std::move(column);
@ -508,10 +506,10 @@ public:
return std::make_shared<DataTypeUInt64>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
size_t current_block_number = block_number++;
block.getByPosition(result).column = ColumnUInt64::create(block.rows(), current_block_number);
block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, current_block_number);
}
};
@ -552,15 +550,14 @@ public:
return std::make_shared<DataTypeUInt64>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
size_t rows_in_block = block.rows();
size_t current_row_number = rows.fetch_add(rows_in_block);
size_t current_row_number = rows.fetch_add(input_rows_count);
auto column = ColumnUInt64::create();
auto & data = column->getData();
data.resize(rows_in_block);
for (size_t i = 0; i < rows_in_block; ++i)
data.resize(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i)
data[i] = current_row_number + i;
block.getByPosition(result).column = std::move(column);
@ -614,7 +611,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn * col = block.getByPosition(arguments[0]).column.get();
@ -671,7 +668,7 @@ public:
return arguments[0];
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto & src = block.getByPosition(arguments[0]).column;
if (ColumnPtr converted = src->convertToFullColumnIfConst())
@ -734,7 +731,7 @@ public:
return false;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
/// Second argument must be ColumnSet.
ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column;
@ -804,9 +801,9 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(block.rows(), UInt64(0));
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(0));
}
};
@ -852,9 +849,9 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(block.rows(), UInt64(1));
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(1));
}
};
@ -883,7 +880,7 @@ public:
return arguments.front();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
block.getByPosition(result).column = block.getByPosition(arguments.front()).column;
}
@ -928,7 +925,7 @@ public:
return arr->getNestedType();
}
void executeImpl(Block & /*block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) override
void executeImpl(Block &, const ColumnNumbers &, size_t, size_t /*input_rows_count*/) override
{
throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL);
}
@ -955,7 +952,7 @@ DataTypePtr FunctionReplicate::getReturnTypeImpl(const DataTypes & arguments) co
return std::make_shared<DataTypeArray>(arguments[0]);
}
void FunctionReplicate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionReplicate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
ColumnPtr first_column = block.getByPosition(arguments[0]).column;
@ -1019,7 +1016,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
Int64 min = extractConstant<Int64>(block, arguments, 1, "Second"); /// The level at which the line has zero length.
Int64 max = extractConstant<Int64>(block, arguments, 2, "Third"); /// The level at which the line has the maximum length.
@ -1145,7 +1142,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto in = block.getByPosition(arguments.front()).column.get();
@ -1274,10 +1271,10 @@ public:
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
static const std::string version = getVersion();
block.getByPosition(result).column = DataTypeString().createColumnConst(block.rows(), version);
block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, version);
}
private:
@ -1317,9 +1314,9 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeUInt32().createColumnConst(block.rows(), static_cast<UInt64>(uptime));
block.getByPosition(result).column = DataTypeUInt32().createColumnConst(input_rows_count, static_cast<UInt64>(uptime));
}
private:
@ -1354,9 +1351,9 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = DataTypeString().createColumnConst(block.rows(), DateLUT::instance().getTimeZone());
block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, DateLUT::instance().getTimeZone());
}
};
@ -1403,7 +1400,7 @@ public:
return type->getReturnType();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnAggregateFunction * column_with_states
= typeid_cast<const ColumnAggregateFunction *>(&*block.getByPosition(arguments.at(0)).column);
@ -1558,7 +1555,7 @@ public:
return res;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
auto & src = block.getByPosition(arguments.at(0));
const auto & res_type = block.getByPosition(result).type;
@ -1566,7 +1563,7 @@ public:
/// When column is constant, its difference is zero.
if (src.column->isColumnConst())
{
block.getByPosition(result).column = res_type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = res_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
@ -1618,7 +1615,7 @@ public:
return type->getReturnType();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnAggregateFunction * column_with_states
= typeid_cast<const ColumnAggregateFunction *>(&*block.getByPosition(arguments.at(0)).column);
@ -1667,17 +1664,17 @@ public:
bool isDeterministic() override { return false; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
const Context & global_context;
};
void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
auto & src = block.getByPosition(arguments[0]);
size_t size = block.rows();
size_t size = input_rows_count;
auto res_col = ColumnUInt64::create(size);
auto & res_data = static_cast<ColumnUInt64 &>(*res_col).getData();
@ -1721,7 +1718,7 @@ DataTypePtr FunctionHasColumnInTable::getReturnTypeImpl(const ColumnsWithTypeAnd
}
void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
auto get_string_from_block = [&](size_t column_pos) -> String
{
@ -1762,7 +1759,7 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers &
has_column = remote_columns.hasPhysical(column_name);
}
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(block.rows(), UInt64(has_column));
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(has_column));
}
@ -1796,7 +1793,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto in = block.getByPosition(arguments.front()).column.get();

View File

@ -32,7 +32,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
@ -59,7 +59,7 @@ public:
return std::const_pointer_cast<FunctionExpression>(shared_from_this());
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
Block expr_block;
for (size_t i = 0; i < arguments.size(); ++i)
@ -143,7 +143,7 @@ public:
return std::const_pointer_cast<FunctionCapture>(shared_from_this());
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
ColumnsWithTypeAndName columns;
columns.reserve(arguments.size());
@ -168,8 +168,7 @@ public:
auto function = std::make_shared<FunctionExpression>(expression_actions, types, names,
function_return_type, expression_return_name);
auto size = block.rows();
block.getByPosition(result).column = ColumnFunction::create(size, std::move(function), columns);
block.getByPosition(result).column = ColumnFunction::create(input_rows_count, std::move(function), columns);
}
size_t getNumberOfArguments() const override { return captured_types.size(); }

View File

@ -7,6 +7,9 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeNothing.h>
#include <Columns/ColumnNullable.h>
#include <cstdlib>
#include <string>
#include <memory>
namespace DB
@ -40,7 +43,7 @@ DataTypePtr FunctionIsNull::getReturnTypeImpl(const DataTypes &) const
return std::make_shared<DataTypeUInt8>();
}
void FunctionIsNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionIsNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const ColumnWithTypeAndName & elem = block.getByPosition(arguments[0]);
if (elem.column->isColumnNullable())
@ -73,7 +76,7 @@ DataTypePtr FunctionIsNotNull::getReturnTypeImpl(const DataTypes &) const
return std::make_shared<DataTypeUInt8>();
}
void FunctionIsNotNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionIsNotNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
Block temp_block
{
@ -90,8 +93,8 @@ void FunctionIsNotNull::executeImpl(Block & block, const ColumnNumbers & argumen
}
};
FunctionIsNull{}.execute(temp_block, {0}, 1);
FunctionNot{}.execute(temp_block, {1}, 2);
FunctionIsNull{}.execute(temp_block, {0}, 1, input_rows_count);
FunctionNot{}.execute(temp_block, {1}, 2, input_rows_count);
block.getByPosition(result).column = std::move(temp_block.getByPosition(2).column);
}
@ -154,7 +157,7 @@ DataTypePtr FunctionCoalesce::getReturnTypeImpl(const DataTypes & arguments) con
return res;
}
void FunctionCoalesce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionCoalesce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
/// coalesce(arg0, arg1, ..., argN) is essentially
/// multiIf(isNotNull(arg0), assumeNotNull(arg0), isNotNull(arg1), assumeNotNull(arg1), ..., argN)
@ -193,9 +196,9 @@ void FunctionCoalesce::executeImpl(Block & block, const ColumnNumbers & argument
else
{
temp_block.insert({nullptr, std::make_shared<DataTypeUInt8>(), ""});
is_not_null.execute(temp_block, {filtered_args[i]}, res_pos);
is_not_null.execute(temp_block, {filtered_args[i]}, res_pos, input_rows_count);
temp_block.insert({nullptr, removeNullable(block.getByPosition(filtered_args[i]).type), ""});
assume_not_null.execute(temp_block, {filtered_args[i]}, res_pos + 1);
assume_not_null.execute(temp_block, {filtered_args[i]}, res_pos + 1, input_rows_count);
multi_if_args.push_back(res_pos);
multi_if_args.push_back(res_pos + 1);
@ -205,7 +208,7 @@ void FunctionCoalesce::executeImpl(Block & block, const ColumnNumbers & argument
/// If all arguments appeared to be NULL.
if (multi_if_args.empty())
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(block.rows());
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
@ -215,7 +218,7 @@ void FunctionCoalesce::executeImpl(Block & block, const ColumnNumbers & argument
return;
}
FunctionMultiIf{context}.execute(temp_block, multi_if_args, result);
FunctionMultiIf{context}.execute(temp_block, multi_if_args, result, input_rows_count);
ColumnPtr res = std::move(temp_block.getByPosition(result).column);
@ -249,7 +252,7 @@ DataTypePtr FunctionIfNull::getReturnTypeImpl(const DataTypes & arguments) const
return FunctionIf{}.getReturnTypeImpl({std::make_shared<DataTypeUInt8>(), removeNullable(arguments[0]), arguments[1]});
}
void FunctionIfNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionIfNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
/// Always null.
if (block.getByPosition(arguments[0]).type->onlyNull())
@ -274,10 +277,10 @@ void FunctionIfNull::executeImpl(Block & block, const ColumnNumbers & arguments,
size_t assume_not_null_pos = temp_block.columns();
temp_block.insert({nullptr, removeNullable(block.getByPosition(arguments[0]).type), ""});
FunctionIsNotNull{}.execute(temp_block, {arguments[0]}, is_not_null_pos);
FunctionAssumeNotNull{}.execute(temp_block, {arguments[0]}, assume_not_null_pos);
FunctionIsNotNull{}.execute(temp_block, {arguments[0]}, is_not_null_pos, input_rows_count);
FunctionAssumeNotNull{}.execute(temp_block, {arguments[0]}, assume_not_null_pos, input_rows_count);
FunctionIf{}.execute(temp_block, {is_not_null_pos, assume_not_null_pos, arguments[1]}, result);
FunctionIf{}.execute(temp_block, {is_not_null_pos, assume_not_null_pos, arguments[1]}, result, input_rows_count);
block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column);
}
@ -299,7 +302,7 @@ DataTypePtr FunctionNullIf::getReturnTypeImpl(const DataTypes & arguments) const
return FunctionIf{}.getReturnTypeImpl({std::make_shared<DataTypeUInt8>(), makeNullable(arguments[0]), arguments[0]});
}
void FunctionNullIf::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionNullIf::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
/// nullIf(col1, col2) == if(col1 == col2, NULL, col1)
@ -308,7 +311,7 @@ void FunctionNullIf::executeImpl(Block & block, const ColumnNumbers & arguments,
size_t res_pos = temp_block.columns();
temp_block.insert({nullptr, std::make_shared<DataTypeUInt8>(), ""});
FunctionEquals{}.execute(temp_block, {arguments[0], arguments[1]}, res_pos);
FunctionEquals{}.execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count);
/// Argument corresponding to the NULL value.
size_t null_pos = temp_block.columns();
@ -316,12 +319,12 @@ void FunctionNullIf::executeImpl(Block & block, const ColumnNumbers & arguments,
/// Append a NULL column.
ColumnWithTypeAndName null_elem;
null_elem.type = block.getByPosition(result).type;
null_elem.column = null_elem.type->createColumnConstWithDefaultValue(temp_block.rows());
null_elem.column = null_elem.type->createColumnConstWithDefaultValue(input_rows_count);
null_elem.name = "NULL";
temp_block.insert(null_elem);
FunctionIf{}.execute(temp_block, {res_pos, null_pos, arguments[0]}, result);
FunctionIf{}.execute(temp_block, {res_pos, null_pos, arguments[0]}, result, input_rows_count);
block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column);
}
@ -343,7 +346,7 @@ DataTypePtr FunctionAssumeNotNull::getReturnTypeImpl(const DataTypes & arguments
return removeNullable(arguments[0]);
}
void FunctionAssumeNotNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionAssumeNotNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const ColumnPtr & col = block.getByPosition(arguments[0]).column;
ColumnPtr & res_col = block.getByPosition(result).column;
@ -374,7 +377,7 @@ DataTypePtr FunctionToNullable::getReturnTypeImpl(const DataTypes & arguments) c
return makeNullable(arguments[0]);
}
void FunctionToNullable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void FunctionToNullable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
block.getByPosition(result).column = makeNullable(block.getByPosition(arguments[0]).column);
}

View File

@ -24,7 +24,7 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
/// Implements the function isNotNull which returns true if a value
@ -40,7 +40,7 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
/// Implements the function coalesce which takes a set of arguments and
@ -58,7 +58,7 @@ public:
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
const Context & context;
@ -78,7 +78,7 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
/// Implements the function nullIf which takes 2 arguments and returns
@ -95,7 +95,7 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
/// Implements the function assumeNotNull which takes 1 argument and works as follows:
@ -113,7 +113,7 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
/// If value is not Nullable or NULL, wraps it to Nullable.
@ -128,7 +128,7 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
}

View File

@ -0,0 +1,199 @@
#include <Functions/FunctionsProjection.h>
#include <DataTypes/DataTypesNumber.h>
#include <memory>
#include <vector>
namespace DB {
FunctionPtr FunctionOneOrZero::create(const Context &)
{
return std::make_shared<FunctionOneOrZero>();
}
String FunctionOneOrZero::getName() const
{
return name;
}
size_t FunctionOneOrZero::getNumberOfArguments() const
{
return 1;
}
DataTypePtr FunctionOneOrZero::getReturnTypeImpl(const DataTypes & /*arguments*/) const
{
return std::make_shared<DataTypeUInt8>();
}
void FunctionOneOrZero::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const auto & data_column = block.getByPosition(arguments[0]).column;
auto col_res = ColumnUInt8::create();
auto & vec_res = col_res->getData();
vec_res.resize(data_column->size());
for (size_t i = 0; i < data_column->size(); ++i)
{
if (data_column->getBoolRepresentation(i))
{
vec_res[i] = 1;
}
else
{
vec_res[i] = 0;
}
}
block.getByPosition(result).column = std::move(col_res);
}
FunctionPtr FunctionProject::create(const Context &)
{
return std::make_shared<FunctionProject>();
}
String FunctionProject::getName() const
{
return name;
}
size_t FunctionProject::getNumberOfArguments() const
{
return 2;
}
DataTypePtr FunctionProject::getReturnTypeImpl(const DataTypes & arguments) const
{
if (!checkAndGetDataType<DataTypeUInt8>(arguments[1].get()))
{
throw Exception("Illegal type " + arguments[1]->getName() + " of 2nd argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return arguments[0];
}
void FunctionProject::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const auto & data_column = block.getByPosition(arguments[0]).column;
const auto & projection_column = block.getByPosition(arguments[1]).column;
if (const auto projection_column_uint8 = checkAndGetColumn<ColumnUInt8>(projection_column.get()))
{
block.getByPosition(result).column = data_column->filter(projection_column_uint8->getData(), -1);
}
else if (const auto projection_column_uint8_const = checkAndGetColumnConst<ColumnUInt8>(projection_column.get()))
{
if (projection_column_uint8_const->getBoolRepresentation(0))
{
block.getByPosition(result).column = data_column->cloneResized(data_column->size());
}
else
{
block.getByPosition(result).column = data_column->cloneEmpty();
}
}
else
{
throw Exception("Unexpected column: " + projection_column->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
}
FunctionPtr FunctionBuildProjectionComposition::create(const Context &)
{
return std::make_shared<FunctionBuildProjectionComposition>();
}
String FunctionBuildProjectionComposition::getName() const
{
return name;
}
size_t FunctionBuildProjectionComposition::getNumberOfArguments() const
{
return 2;
}
DataTypePtr FunctionBuildProjectionComposition::getReturnTypeImpl(const DataTypes & arguments) const
{
for (size_t i = 0; i < 2; ++i)
{
if (!checkAndGetDataType<DataTypeUInt8>(arguments[i].get()))
{
throw Exception("Illegal type " + arguments[i]->getName() + " of " + std::to_string(i + 1) + " argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
return std::make_shared<DataTypeUInt8>();
}
void FunctionBuildProjectionComposition::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const auto & first_projection_column = block.getByPosition(arguments[0]).column;
const auto & second_projection_column = block.getByPosition(arguments[1]).column;
auto col_res = ColumnUInt8::create();
auto & vec_res = col_res->getData();
vec_res.resize(first_projection_column->size());
size_t current_reserve_index = 0;
for (size_t i = 0; i < first_projection_column->size(); ++i)
{
if (first_projection_column->getBoolRepresentation(i) == 0)
{
vec_res[i] = 0;
}
else
{
vec_res[i] = second_projection_column->getBoolRepresentation(current_reserve_index++);
}
}
if (current_reserve_index != second_projection_column->size())
{
throw Exception("Second argument size is not appropriate: " + std::to_string(second_projection_column->size())
+ " instead of " + std::to_string(current_reserve_index),
ErrorCodes::BAD_ARGUMENTS);
}
block.getByPosition(result).column = std::move(col_res);
}
FunctionPtr FunctionRestoreProjection::create(const Context &)
{
return std::make_shared<FunctionRestoreProjection>();
}
String FunctionRestoreProjection::getName() const
{
return name;
}
bool FunctionRestoreProjection::isVariadic() const {
return true;
}
size_t FunctionRestoreProjection::getNumberOfArguments() const
{
return 0;
}
DataTypePtr FunctionRestoreProjection::getReturnTypeImpl(const DataTypes & arguments) const
{
if (arguments.size() < 2)
{
throw Exception("Wrong argument count: " + std::to_string(arguments.size()), ErrorCodes::BAD_ARGUMENTS);
}
return arguments[1];
}
void FunctionRestoreProjection::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
if (arguments.size() < 2)
{
throw Exception("Wrong argument count: " + std::to_string(arguments.size()), ErrorCodes::BAD_ARGUMENTS);
}
const auto & projection_column = block.getByPosition(arguments[0]).column;
auto col_res = block.getByPosition(arguments[1]).column->cloneEmpty();
std::vector<size_t> override_indices(arguments.size() - 1, 0);
for (size_t i = 0; i < projection_column->size(); ++i)
{
size_t argument_index = projection_column->getBoolRepresentation(i);
col_res->insertFrom(*block.getByPosition(arguments[argument_index + 1]).column, override_indices[argument_index]++);
}
block.getByPosition(result).column = std::move(col_res);
}
}

View File

@ -0,0 +1,67 @@
#pragma once
#include <Functions/IFunction.h>
#include "FunctionsConversion.h"
namespace DB {
/*
* This function accepts one column and converts it to UInt8, replacing values, which evaluate to true, with 1, and values,
* which evaluate to false with 0
*/
class FunctionOneOrZero final : public IFunction {
public:
static constexpr auto name = "one_or_zero";
static FunctionPtr create(const Context &);
String getName() const override;
size_t getNumberOfArguments() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
/*
* FunctionProject accepts two columns: data column and projection column.
* Projection column is a column of UInt8 values 0 and 1, which indicate the binary mask of rows, where to project.
* This function builds a column of a smaller, which contains values of the data column at the positions where
* the projection column contained 1. The size of result column equals the count of ones in the projection column.
*/
class FunctionProject final : public IFunction {
public:
static constexpr auto name = "__inner_project__";
static FunctionPtr create(const Context &);
String getName() const override;
size_t getNumberOfArguments() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
/*
* FunctionBuildProjectionComposition constructs the composition of two projection columns. The size of
* second projection column should equal the count of ones in the first input projection column.
*/
class FunctionBuildProjectionComposition final : public IFunction {
public:
static constexpr auto name = "__inner_build_projection_composition__";
static FunctionPtr create(const Context &);
String getName() const override;
size_t getNumberOfArguments() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
/*
* Accepts mapping column with values from range [0, N) and N more columns as arguments.
* Forms a column by taking value from column, which number is in the mapping column.
*/
class FunctionRestoreProjection final : public IFunction {
public:
static constexpr auto name = "__inner_restore_projection__";
static FunctionPtr create(const Context &);
String getName() const override;
bool isVariadic() const override;
size_t getNumberOfArguments() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
};
}

View File

@ -150,12 +150,12 @@ public:
return std::make_shared<DataTypeNumber<typename Impl::ReturnType>>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
auto col_to = ColumnVector<ToType>::create();
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
size_t size = block.rows();
size_t size = input_rows_count;
vec_to.resize(size);
Impl::execute(&vec_to[0], vec_to.size());
@ -196,7 +196,7 @@ public:
return std::make_shared<DataTypeNumber<typename Impl::ReturnType>>();
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
if (!is_initialized)
{
@ -206,7 +206,7 @@ public:
value = vec_to[0];
}
block.getByPosition(result).column = DataTypeNumber<ToType>().createColumnConst(block.rows(), toField(value));
block.getByPosition(result).column = DataTypeNumber<ToType>().createColumnConst(input_rows_count, toField(value));
}
};

View File

@ -78,7 +78,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn & src = *block.getByPosition(arguments[0]).column;
MutableColumnPtr dst = block.getByPosition(result).type->createColumn();
@ -133,7 +133,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const IColumn & src = *block.getByPosition(arguments[0]).column;
MutableColumnPtr dst = block.getByPosition(result).type->createColumn();
@ -175,7 +175,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (const ColumnString * col_from = typeid_cast<const ColumnString *>(block.getByPosition(arguments[0]).column.get()))
{

View File

@ -572,7 +572,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (!( executeForType<UInt8>(block, arguments, result)
|| executeForType<UInt16>(block, arguments, result)

View File

@ -553,7 +553,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
@ -641,7 +641,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
@ -658,7 +658,7 @@ public:
}
else if (checkColumn<ColumnArray>(column.get()))
{
FunctionArrayReverse().execute(block, arguments, result);
FunctionArrayReverse().execute(block, arguments, result, input_rows_count);
}
else
throw Exception(
@ -723,21 +723,21 @@ public:
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
if (!is_injective && !arguments.empty() && checkDataType<DataTypeArray>(block.getByPosition(arguments[0]).type.get()))
return FunctionArrayConcat(context).executeImpl(block, arguments, result);
return FunctionArrayConcat(context).executeImpl(block, arguments, result, input_rows_count);
if (arguments.size() == 2)
executeBinary(block, arguments, result);
executeBinary(block, arguments, result, input_rows_count);
else
executeNAry(block, arguments, result);
executeNAry(block, arguments, result, input_rows_count);
}
private:
const Context & context;
void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result)
void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
const IColumn * c0 = block.getByPosition(arguments[0]).column.get();
const IColumn * c1 = block.getByPosition(arguments[1]).column.get();
@ -758,14 +758,14 @@ private:
else
{
/// Fallback: use generic implementation for not very important cases.
executeNAry(block, arguments, result);
executeNAry(block, arguments, result, input_rows_count);
return;
}
block.getByPosition(result).column = std::move(c_res);
}
void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result)
void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
size_t num_sources = arguments.size();
StringSources sources(num_sources);
@ -774,7 +774,7 @@ private:
sources[i] = createDynamicStringSource(*block.getByPosition(arguments[i]).column);
auto c_res = ColumnString::create();
concat(sources, StringSink(*c_res, block.rows()));
concat(sources, StringSink(*c_res, input_rows_count));
block.getByPosition(result).column = std::move(c_res);
}
};
@ -827,12 +827,10 @@ public:
}
template <typename Source>
void executeForSource(
const ColumnPtr & column_start, const ColumnPtr & column_length,
const ColumnConst * column_start_const, const ColumnConst * column_length_const,
Int64 start_value, Int64 length_value,
Block & block, size_t result,
Source && source)
void executeForSource(const ColumnPtr & column_start, const ColumnPtr & column_length,
const ColumnConst * column_start_const, const ColumnConst * column_length_const,
Int64 start_value, Int64 length_value, Block & block, size_t result, Source && source,
size_t input_rows_count)
{
auto col_res = ColumnString::create();
@ -841,34 +839,34 @@ public:
if (column_start_const)
{
if (start_value > 0)
sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, block.rows()), start_value - 1);
sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), start_value - 1);
else if (start_value < 0)
sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, block.rows()), -start_value);
sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -start_value);
else
throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
}
else
sliceDynamicOffsetUnbounded(source, StringSink(*col_res, block.rows()), *column_start);
sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start);
}
else
{
if (column_start_const && column_length_const)
{
if (start_value > 0)
sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, block.rows()), start_value - 1, length_value);
sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), start_value - 1, length_value);
else if (start_value < 0)
sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, block.rows()), -start_value, length_value);
sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -start_value, length_value);
else
throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
}
else
sliceDynamicOffsetBounded(source, StringSink(*col_res, block.rows()), *column_start, *column_length);
sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_start, *column_length);
}
block.getByPosition(result).column = std::move(col_res);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
size_t number_of_arguments = arguments.size();
@ -900,17 +898,17 @@ public:
}
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_string.get()))
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value,
block, result, StringSource(*col));
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, block, result, StringSource(*col), input_rows_count);
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column_string.get()))
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value,
block, result, FixedStringSource(*col));
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, block, result, FixedStringSource(*col), input_rows_count);
else if (const ColumnConst * col = checkAndGetColumnConst<ColumnString>(column_string.get()))
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value,
block, result, ConstSource<StringSource>(*col));
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, block, result, ConstSource<StringSource>(*col), input_rows_count);
else if (const ColumnConst * col = checkAndGetColumnConst<ColumnFixedString>(column_string.get()))
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value,
block, result, ConstSource<FixedStringSource>(*col));
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
length_value, block, result, ConstSource<FixedStringSource>(*col), input_rows_count);
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
@ -956,7 +954,7 @@ public:
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column_string = block.getByPosition(arguments[0]).column;
const ColumnPtr column_start = block.getByPosition(arguments[1]).column;
@ -1032,7 +1030,7 @@ private:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const auto & column = block.getByPosition(arguments[0]).column;
const auto & column_char = block.getByPosition(arguments[1]).column;

View File

@ -165,7 +165,7 @@ public:
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))

View File

@ -339,7 +339,7 @@ public:
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
Generator generator;
generator.init(block, arguments);
@ -514,7 +514,7 @@ public:
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
String delimiter;
if (arguments.size() == 2)

View File

@ -962,7 +962,7 @@ public:
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column_src = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;

View File

@ -72,7 +72,7 @@ public:
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
using ResultType = typename Impl::ResultType;
@ -156,7 +156,7 @@ public:
return std::make_shared<DataTypeString>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;

View File

@ -143,7 +143,7 @@ public:
}
}
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const ColumnConst * array_from = checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[1]).column.get());
const ColumnConst * array_to = checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[2]).column.get());
@ -157,7 +157,7 @@ public:
if (in->isColumnConst())
{
executeConst(block, arguments, result);
executeConst(block, arguments, result, input_rows_count);
return;
}
@ -189,7 +189,7 @@ public:
}
private:
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result)
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
/// Materialize the input column and compute the function as usual.
@ -197,7 +197,7 @@ private:
ColumnNumbers tmp_arguments;
tmp_block.insert(block.getByPosition(arguments[0]));
tmp_block.getByPosition(0).column = tmp_block.getByPosition(0).column->cloneResized(block.rows())->convertToFullColumnIfConst();
tmp_block.getByPosition(0).column = tmp_block.getByPosition(0).column->cloneResized(input_rows_count)->convertToFullColumnIfConst();
tmp_arguments.push_back(0);
for (size_t i = 1; i < arguments.size(); ++i)
@ -209,7 +209,7 @@ private:
tmp_block.insert(block.getByPosition(result));
size_t tmp_result = arguments.size();
execute(tmp_block, tmp_arguments, tmp_result);
execute(tmp_block, tmp_arguments, tmp_result, input_rows_count);
block.getByPosition(result).column = tmp_block.getByPosition(tmp_result).column;
}

View File

@ -9,6 +9,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Interpreters/ExpressionActions.h>
#include <memory>
namespace DB
@ -66,7 +67,7 @@ public:
return std::make_shared<DataTypeTuple>(arguments);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
size_t tuple_size = arguments.size();
Columns tuple_columns(tuple_size);
@ -142,7 +143,7 @@ public:
return out_return_type;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
Columns array_offsets;

View File

@ -8,6 +8,8 @@
#include <Common/typeid_cast.h>
#include <ext/range.h>
#include <ext/collection_cast.h>
#include <cstdlib>
#include <memory>
namespace DB
@ -26,7 +28,7 @@ namespace
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in blocks.
* Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL.
*/
ColumnPtr wrapInNullable(const ColumnPtr & src, Block & block, const ColumnNumbers & args, size_t result)
ColumnPtr wrapInNullable(const ColumnPtr & src, Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
ColumnPtr result_null_map_column;
@ -49,7 +51,7 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, Block & block, const ColumnNumbe
/// Const Nullable that are NULL.
if (elem.column->onlyNull())
return block.getByPosition(result).type->createColumnConst(block.rows(), Null());
return block.getByPosition(result).type->createColumnConst(input_rows_count, Null());
if (elem.column->isColumnConst())
continue;
@ -134,7 +136,8 @@ bool allArgumentsAreConstants(const Block & block, const ColumnNumbers & args)
}
}
bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result)
bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count)
{
ColumnNumbers arguments_to_remain_constants = getArgumentsThatAreAlwaysConstant();
@ -176,14 +179,15 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
for (size_t i = 0; i < arguments_size; ++i)
temporary_argument_numbers[i] = i;
execute(temporary_block, temporary_argument_numbers, arguments_size);
execute(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows());
block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, block.rows());
block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, input_rows_count);
return true;
}
bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result)
bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count)
{
if (args.empty() || !useDefaultImplementationForNulls())
return false;
@ -192,30 +196,31 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co
if (null_presence.has_null_constant)
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(block.rows(), Null());
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, Null());
return true;
}
if (null_presence.has_nullable)
{
Block temporary_block = createBlockWithNestedColumns(block, args, result);
execute(temporary_block, args, result);
block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args, result);
execute(temporary_block, args, result, temporary_block.rows());
block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args,
result, input_rows_count);
return true;
}
return false;
}
void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result)
void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
if (defaultImplementationForConstantArguments(block, args, result))
if (defaultImplementationForConstantArguments(block, args, result, input_rows_count))
return;
if (defaultImplementationForNulls(block, args, result))
if (defaultImplementationForNulls(block, args, result, input_rows_count))
return;
executeImpl(block, args, result);
executeImpl(block, args, result, input_rows_count);
}
void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const

View File

@ -31,7 +31,7 @@ public:
/// Get the main function name.
virtual String getName() const = 0;
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result) = 0;
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0;
};
using PreparedFunctionPtr = std::shared_ptr<IPreparedFunction>;
@ -39,10 +39,10 @@ using PreparedFunctionPtr = std::shared_ptr<IPreparedFunction>;
class PreparedFunctionImpl : public IPreparedFunction
{
public:
void execute(Block & block, const ColumnNumbers & arguments, size_t result) final;
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final;
protected:
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) = 0;
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0;
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
* if some of arguments are NULL constants then return NULL constant,
@ -64,8 +64,10 @@ protected:
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
private:
bool defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result);
bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result);
bool defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count);
bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count);
};
/// Function with known arguments and return type.
@ -85,9 +87,9 @@ public:
virtual PreparedFunctionPtr prepare(const Block & sample_block) const = 0;
/// TODO: make const
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result)
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
return prepare(block)->execute(block, arguments, result);
return prepare(block)->execute(block, arguments, result, input_rows_count);
}
/** Should we evaluate this function while constant folding, if arguments are constants?
@ -249,7 +251,7 @@ class IFunction : public std::enable_shared_from_this<IFunction>,
public:
String getName() const override = 0;
/// TODO: make const
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override = 0;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override = 0;
/// Override this functions to change default implementation behavior. See details in IMyFunction.
bool useDefaultImplementationForNulls() const override { return true; }
@ -294,9 +296,9 @@ public:
String getName() const override { return function->getName(); }
protected:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) final
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
{
return function->executeImpl(block, arguments, result);
return function->executeImpl(block, arguments, result, input_rows_count);
}
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }

View File

@ -3,7 +3,6 @@
#include <Functions/FunctionFactory.h>
#include <iostream>
namespace DB
{
@ -42,7 +41,7 @@ void registerFunctionsGeo(FunctionFactory &);
void registerFunctionsCharset(FunctionFactory &);
void registerFunctionsNull(FunctionFactory &);
void registerFunctionsFindCluster(FunctionFactory &);
void registerFunctionsProjection(FunctionFactory &);
void registerFunctions()
{
@ -79,6 +78,7 @@ void registerFunctions()
registerFunctionsCharset(factory);
registerFunctionsNull(factory);
registerFunctionsFindCluster(factory);
registerFunctionsProjection(factory);
}
}

View File

@ -44,13 +44,19 @@ Names ExpressionAction::getNeededColumns() const
if (!source_name.empty())
res.push_back(source_name);
if (!row_projection_column.empty())
{
res.push_back(row_projection_column);
}
return res;
}
ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & function_,
const std::vector<std::string> & argument_names_,
std::string result_name_)
std::string result_name_,
const std::string & row_projection_column)
{
if (result_name_ == "")
{
@ -69,16 +75,21 @@ ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & func
a.result_name = result_name_;
a.function_builder = function_;
a.argument_names = argument_names_;
a.row_projection_column = row_projection_column;
return a;
}
ExpressionAction ExpressionAction::addColumn(const ColumnWithTypeAndName & added_column_)
ExpressionAction ExpressionAction::addColumn(const ColumnWithTypeAndName & added_column_,
const std::string & row_projection_column,
bool is_row_projection_complementary)
{
ExpressionAction a;
a.type = ADD_COLUMN;
a.result_name = added_column_.name;
a.result_type = added_column_.type;
a.added_column = added_column_.column;
a.row_projection_column = row_projection_column;
a.is_row_projection_complementary = is_row_projection_complementary;
return a;
}
@ -179,7 +190,7 @@ void ExpressionAction::prepare(Block & sample_block)
new_column.type = result_type;
sample_block.insert(std::move(new_column));
function->execute(sample_block, arguments, result_position);
function->execute(sample_block, arguments, result_position, sample_block.rows());
/// If the result is not a constant, just in case, we will consider the result as unknown.
ColumnWithTypeAndName & col = sample_block.safeGetByPosition(result_position);
@ -275,11 +286,45 @@ void ExpressionAction::prepare(Block & sample_block)
}
}
size_t ExpressionAction::getInputRowsCount(Block & block, std::unordered_map<std::string, size_t> & input_rows_counts) const {
auto it = input_rows_counts.find(row_projection_column);
size_t projection_space_dimention;
if (it == input_rows_counts.end())
{
const auto & projection_column = block.getByName(row_projection_column).column;
projection_space_dimention = 0;
for (size_t i = 0; i < projection_column->size(); ++i)
{
if (projection_column->getBoolRepresentation(i) > 0)
{
++projection_space_dimention;
}
}
void ExpressionAction::execute(Block & block) const
input_rows_counts[row_projection_column] = projection_space_dimention;
}
else
{
projection_space_dimention = it->second;
}
size_t parent_space_dimention;
if (row_projection_column.empty())
{
parent_space_dimention = input_rows_counts[""];
}
else
{
parent_space_dimention = block.getByName(row_projection_column).column->size();
}
return is_row_projection_complementary ? parent_space_dimention - projection_space_dimention : projection_space_dimention;
}
void ExpressionAction::execute(Block & block, std::unordered_map<std::string, size_t> & input_rows_counts) const
{
// std::cerr << "executing: " << toString() << std::endl;
size_t input_rows_count = getInputRowsCount(block, input_rows_counts);
if (type == REMOVE_COLUMN || type == COPY_COLUMN)
if (!block.has(source_name))
throw Exception("Not found column '" + source_name + "'. There are columns: " + block.dumpNames(), ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
@ -304,7 +349,7 @@ void ExpressionAction::execute(Block & block) const
block.insert({ nullptr, result_type, result_name});
ProfileEvents::increment(ProfileEvents::FunctionExecute);
function->execute(block, arguments, num_columns_without_result);
function->execute(block, arguments, num_columns_without_result, input_rows_count);
break;
}
@ -332,7 +377,7 @@ void ExpressionAction::execute(Block & block) const
Block tmp_block{src_col, {{}, src_col.type, {}}};
function_builder->build({src_col})->execute(tmp_block, {0}, 1);
function_builder->build({src_col})->execute(tmp_block, {0}, 1, src_col.column->size());
non_empty_array_columns[name] = tmp_block.safeGetByPosition(1).column;
}
@ -371,6 +416,8 @@ void ExpressionAction::execute(Block & block) const
}
}
// Temporary support case with no projections
input_rows_counts[""] = block.rows();
break;
}
@ -404,7 +451,7 @@ void ExpressionAction::execute(Block & block) const
break;
case ADD_COLUMN:
block.insert({ added_column->cloneResized(block.rows()), result_type, result_name });
block.insert({ added_column->cloneResized(input_rows_count), result_type, result_name });
break;
case COPY_COLUMN:
@ -419,8 +466,10 @@ void ExpressionAction::execute(Block & block) const
void ExpressionAction::executeOnTotals(Block & block) const
{
std::unordered_map<std::string, size_t> input_rows_counts;
input_rows_counts[""] = block.rows();
if (type != JOIN)
execute(block);
execute(block, input_rows_counts);
else
join->joinTotals(block);
}
@ -629,9 +678,11 @@ bool ExpressionActions::popUnusedArrayJoin(const Names & required_columns, Expre
void ExpressionActions::execute(Block & block) const
{
std::unordered_map<std::string, size_t> input_rows_counts;
input_rows_counts[""] = block.rows();
for (const auto & action : actions)
{
action.execute(block);
action.execute(block, input_rows_counts);
checkLimits(block);
}
}
@ -838,6 +889,9 @@ void ExpressionActions::finalize(const Names & output_columns)
if (!action.source_name.empty())
++columns_refcount[action.source_name];
if (!action.row_projection_column.empty())
++columns_refcount[action.row_projection_column];
for (const auto & name : action.argument_names)
++columns_refcount[name];
@ -866,6 +920,9 @@ void ExpressionActions::finalize(const Names & output_columns)
if (!action.source_name.empty())
process(action.source_name);
if (!action.row_projection_column.empty())
process(action.row_projection_column);
for (const auto & name : action.argument_names)
process(name);

View File

@ -67,6 +67,10 @@ public:
std::string result_name;
DataTypePtr result_type;
/// For conditional projections (projections on subset of rows)
std::string row_projection_column;
bool is_row_projection_complementary = false;
/// For ADD_COLUMN.
ColumnPtr added_column;
@ -88,9 +92,12 @@ public:
/// If result_name_ == "", as name "function_name(arguments separated by commas) is used".
static ExpressionAction applyFunction(
const FunctionBuilderPtr & function_, const std::vector<std::string> & argument_names_, std::string result_name_ = "");
const FunctionBuilderPtr & function_, const std::vector<std::string> & argument_names_, std::string result_name_ = "",
const std::string & row_projection_column = "");
static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_);
static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_,
const std::string & row_projection_column,
bool is_row_projection_complementary);
static ExpressionAction removeColumn(const std::string & removed_name);
static ExpressionAction copyColumn(const std::string & from_name, const std::string & to_name);
static ExpressionAction project(const NamesWithAliases & projected_columns_);
@ -107,7 +114,8 @@ private:
friend class ExpressionActions;
void prepare(Block & sample_block);
void execute(Block & block) const;
size_t getInputRowsCount(Block & block, std::unordered_map<std::string, size_t> & input_rows_counts) const;
void execute(Block & block, std::unordered_map<std::string, size_t> & input_rows_counts) const;
void executeOnTotals(Block & block) const;
};

View File

@ -58,6 +58,7 @@
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeFunction.h>
#include <Functions/FunctionsMiscellaneous.h>
#include "ProjectionManipulation.h"
namespace DB
@ -84,6 +85,7 @@ namespace ErrorCodes
extern const int TOO_DEEP_AST;
extern const int TOO_BIG_AST;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND;
}
@ -1730,114 +1732,106 @@ static String getUniqueName(const Block & block, const String & prefix)
return prefix + toString(i);
}
/** For getActionsImpl.
* A stack of ExpressionActions corresponding to nested lambda expressions.
* The new action should be added to the highest possible level.
* For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)"
* calculation of the product must be done outside the lambda expression (it does not depend on x), and the calculation of the sum is inside (depends on x).
*/
struct ExpressionAnalyzer::ScopeStack
ScopeStack::ScopeStack(const ExpressionActionsPtr & actions, const Settings & settings_)
: settings(settings_)
{
struct Level
stack.emplace_back();
stack.back().actions = actions;
const Block & sample_block = actions->getSampleBlock();
for (size_t i = 0, size = sample_block.columns(); i < size; ++i)
stack.back().new_columns.insert(sample_block.getByPosition(i).name);
}
void ScopeStack::pushLevel(const NamesAndTypesList & input_columns)
{
stack.emplace_back();
Level & prev = stack[stack.size() - 2];
ColumnsWithTypeAndName all_columns;
NameSet new_names;
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
{
ExpressionActionsPtr actions;
NameSet new_columns;
};
using Levels = std::vector<Level>;
Levels stack;
Settings settings;
ScopeStack(const ExpressionActionsPtr & actions, const Settings & settings_)
: settings(settings_)
{
stack.emplace_back();
stack.back().actions = actions;
const Block & sample_block = actions->getSampleBlock();
for (size_t i = 0, size = sample_block.columns(); i < size; ++i)
stack.back().new_columns.insert(sample_block.getByPosition(i).name);
all_columns.emplace_back(nullptr, it->type, it->name);
new_names.insert(it->name);
stack.back().new_columns.insert(it->name);
}
void pushLevel(const NamesAndTypesList & input_columns)
const Block & prev_sample_block = prev.actions->getSampleBlock();
for (size_t i = 0, size = prev_sample_block.columns(); i < size; ++i)
{
stack.emplace_back();
Level & prev = stack[stack.size() - 2];
ColumnsWithTypeAndName all_columns;
NameSet new_names;
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
{
all_columns.emplace_back(nullptr, it->type, it->name);
new_names.insert(it->name);
stack.back().new_columns.insert(it->name);
}
const Block & prev_sample_block = prev.actions->getSampleBlock();
for (size_t i = 0, size = prev_sample_block.columns(); i < size; ++i)
{
const ColumnWithTypeAndName & col = prev_sample_block.getByPosition(i);
if (!new_names.count(col.name))
all_columns.push_back(col);
}
stack.back().actions = std::make_shared<ExpressionActions>(all_columns, settings);
const ColumnWithTypeAndName & col = prev_sample_block.getByPosition(i);
if (!new_names.count(col.name))
all_columns.push_back(col);
}
size_t getColumnLevel(const std::string & name)
stack.back().actions = std::make_shared<ExpressionActions>(all_columns, settings);
}
size_t ScopeStack::getColumnLevel(const std::string & name)
{
for (int i = static_cast<int>(stack.size()) - 1; i >= 0; --i)
if (stack[i].new_columns.count(name))
return i;
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
}
void ScopeStack::addAction(const ExpressionAction & action)
{
size_t level = 0;
Names required = action.getNeededColumns();
for (size_t i = 0; i < required.size(); ++i)
level = std::max(level, getColumnLevel(required[i]));
Names added;
stack[level].actions->add(action, added);
stack[level].new_columns.insert(added.begin(), added.end());
for (size_t i = 0; i < added.size(); ++i)
{
for (int i = static_cast<int>(stack.size()) - 1; i >= 0; --i)
if (stack[i].new_columns.count(name))
return i;
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
const ColumnWithTypeAndName & col = stack[level].actions->getSampleBlock().getByName(added[i]);
for (size_t j = level + 1; j < stack.size(); ++j)
stack[j].actions->addInput(col);
}
}
void addAction(const ExpressionAction & action)
{
size_t level = 0;
Names required = action.getNeededColumns();
for (size_t i = 0; i < required.size(); ++i)
level = std::max(level, getColumnLevel(required[i]));
Names added;
stack[level].actions->add(action, added);
stack[level].new_columns.insert(added.begin(), added.end());
for (size_t i = 0; i < added.size(); ++i)
{
const ColumnWithTypeAndName & col = stack[level].actions->getSampleBlock().getByName(added[i]);
for (size_t j = level + 1; j < stack.size(); ++j)
stack[j].actions->addInput(col);
}
}
ExpressionActionsPtr popLevel()
{
ExpressionActionsPtr res = stack.back().actions;
stack.pop_back();
return res;
}
const Block & getSampleBlock() const
{
return stack.back().actions->getSampleBlock();
}
};
ExpressionActionsPtr ScopeStack::popLevel()
{
ExpressionActionsPtr res = stack.back().actions;
stack.pop_back();
return res;
}
const Block & ScopeStack::getSampleBlock() const
{
return stack.back().actions->getSampleBlock();
}
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions)
{
ScopeStack scopes(actions, settings);
getActionsImpl(ast, no_subqueries, only_consts, scopes);
ProjectionManipulatorPtr projection_manipulator;
if (!isThereArrayJoin(ast) && settings.enable_conditional_computation && !only_consts)
{
projection_manipulator = std::make_shared<ConditionalTree>(scopes, context);
}
else
{
projection_manipulator = std::make_shared<DefaultProjectionManipulator>(scopes);
}
getActionsImpl(ast, no_subqueries, only_consts, scopes, projection_manipulator);
actions = scopes.popLevel();
}
void ExpressionAnalyzer::getArrayJoinedColumns()
{
if (select_query && select_query->array_join_expression_list())
@ -1943,18 +1937,68 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(const ASTPtr & ast)
}
}
bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast)
{
if (typeid_cast<ASTIdentifier *>(ast.get()))
{
return false;
}
else if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
{
if (node->name == "arrayJoin")
{
return true;
}
if (functionIsInOrGlobalInOperator(node->name))
{
return isThereArrayJoin(node->arguments->children.at(0));
}
if (node->name == "indexHint")
{
return false;
}
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
{
return false;
}
for (auto & child : node->arguments->children)
{
if (isThereArrayJoin(child))
{
return true;
}
}
return false;
}
else if (typeid_cast<ASTLiteral *>(ast.get()))
{
return false;
}
else
{
for (auto & child : ast->children)
{
if (isThereArrayJoin(child))
{
return true;
}
}
return false;
}
}
void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack)
void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack,
ProjectionManipulatorPtr projection_manipulator)
{
/// If the result of the calculation already exists in the block.
if ((typeid_cast<ASTFunction *>(ast.get()) || typeid_cast<ASTLiteral *>(ast.get()))
&& actions_stack.getSampleBlock().has(ast->getColumnName()))
&& projection_manipulator->tryToGetFromUpperProjection(ast->getColumnName()))
return;
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(ast.get()))
{
std::string name = node->getColumnName();
if (!only_consts && !actions_stack.getSampleBlock().has(name))
if (!only_consts && !projection_manipulator->tryToGetFromUpperProjection(ast->getColumnName()))
{
/// The requested column is not in the block.
/// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY.
@ -1981,11 +2025,11 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
ASTPtr arg = node->arguments->children.at(0);
getActionsImpl(arg, no_subqueries, only_consts, actions_stack);
getActionsImpl(arg, no_subqueries, only_consts, actions_stack, projection_manipulator);
if (!only_consts)
{
String result_name = node->getColumnName();
actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name));
String result_name = projection_manipulator->getColumnName(node->getColumnName());
actions_stack.addAction(ExpressionAction::copyColumn(projection_manipulator->getColumnName(arg->getColumnName()), result_name));
NameSet joined_columns;
joined_columns.insert(result_name);
actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context));
@ -1999,7 +2043,8 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
if (!no_subqueries)
{
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack);
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack,
projection_manipulator);
/// Transform tuple or subquery into a set.
makeSet(node, actions_stack.getSampleBlock());
@ -2011,10 +2056,11 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
/// We are in the part of the tree that we are not going to compute. You just need to define types.
/// Do not subquery and create sets. We insert an arbitrary column of the correct type.
ColumnWithTypeAndName fake_column;
fake_column.name = node->getColumnName();
fake_column.name = projection_manipulator->getColumnName(node->getColumnName());
fake_column.type = std::make_shared<DataTypeUInt8>();
actions_stack.addAction(ExpressionAction::addColumn(fake_column));
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack);
actions_stack.addAction(ExpressionAction::addColumn(fake_column, projection_manipulator->getProjectionSourceColumn(), false));
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack,
projection_manipulator);
}
return;
}
@ -2025,7 +2071,8 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
if (node->name == "indexHint")
{
actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(), node->getColumnName())));
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(),
projection_manipulator->getColumnName(node->getColumnName())), projection_manipulator->getProjectionSourceColumn(), false));
return;
}
@ -2033,6 +2080,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
return;
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, context);
auto projection_action = getProjectionAction(node->name, actions_stack, projection_manipulator, node->getColumnName(), context);
Names argument_names;
DataTypes argument_types;
@ -2074,11 +2122,13 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
else
column.name = child->getColumnName();
column.name = projection_manipulator->getColumnName(column.name);
if (!actions_stack.getSampleBlock().has(column.name))
{
column.column = ColumnSet::create(1, set);
actions_stack.addAction(ExpressionAction::addColumn(column));
actions_stack.addAction(ExpressionAction::addColumn(column, projection_manipulator->getProjectionSourceColumn(), false));
}
argument_types.push_back(column.type);
@ -2087,8 +2137,11 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
else
{
/// If the argument is not a lambda expression, call it recursively and find out its type.
getActionsImpl(child, no_subqueries, only_consts, actions_stack);
std::string name = child->getColumnName();
projection_action->preArgumentAction();
getActionsImpl(child, no_subqueries, only_consts, actions_stack,
projection_manipulator);
std::string name = projection_manipulator->getColumnName(child->getColumnName());
projection_action->postArgumentAction(child->getColumnName());
if (actions_stack.getSampleBlock().has(name))
{
argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type);
@ -2102,7 +2155,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
}
else
{
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
throw Exception("Unknown identifier: " + name + ", projection layer " + projection_manipulator->getProjectionExpression() , ErrorCodes::UNKNOWN_IDENTIFIER);
}
}
}
@ -2139,11 +2192,13 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]);
}
projection_action->preArgumentAction();
actions_stack.pushLevel(lambda_arguments);
getActionsImpl(lambda->arguments->children.at(1), no_subqueries, only_consts, actions_stack);
getActionsImpl(lambda->arguments->children.at(1), no_subqueries, only_consts, actions_stack,
projection_manipulator);
ExpressionActionsPtr lambda_actions = actions_stack.popLevel();
String result_name = lambda->arguments->children.at(1)->getColumnName();
String result_name = projection_manipulator->getColumnName(lambda->arguments->children.at(1)->getColumnName());
lambda_actions->finalize(Names(1, result_name));
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
@ -2159,10 +2214,12 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
auto function_capture = std::make_shared<FunctionCapture>(
lambda_actions, captured, lambda_arguments, result_type, result_name);
actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name));
actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name,
projection_manipulator->getProjectionSourceColumn()));
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);
argument_names[i] = lambda_name;
projection_action->postArgumentAction(lambda_name);
}
}
}
@ -2180,7 +2237,17 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
}
if (arguments_present)
actions_stack.addAction(ExpressionAction::applyFunction(function_builder, argument_names, node->getColumnName()));
{
projection_action->preCalculation();
if (projection_action->isCalculationRequired())
{
actions_stack.addAction(
ExpressionAction::applyFunction(function_builder,
argument_names,
projection_manipulator->getColumnName(node->getColumnName()),
projection_manipulator->getProjectionSourceColumn()));
}
}
}
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(ast.get()))
{
@ -2191,7 +2258,8 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
column.type = type;
column.name = node->getColumnName();
actions_stack.addAction(ExpressionAction::addColumn(column));
actions_stack.addAction(ExpressionAction::addColumn(column, "", false));
projection_manipulator->tryToGetFromUpperProjection(column.name);
}
else
{
@ -2200,7 +2268,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
/// Do not go to FROM, JOIN, UNION.
if (!typeid_cast<const ASTTableExpression *>(child.get())
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
getActionsImpl(child, no_subqueries, only_consts, actions_stack);
getActionsImpl(child, no_subqueries, only_consts, actions_stack, projection_manipulator);
}
}
}

View File

@ -3,6 +3,8 @@
#include <Interpreters/AggregateDescription.h>
#include <Interpreters/Settings.h>
#include <Core/Block.h>
#include "ExpressionActions.h"
#include "ProjectionManipulation.h"
namespace DB
@ -54,6 +56,31 @@ struct SubqueryForSet
/// ID of subquery -> what to do with it.
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
struct ScopeStack
{
struct Level
{
ExpressionActionsPtr actions;
NameSet new_columns;
};
using Levels = std::vector<Level>;
Levels stack;
Settings settings;
ScopeStack(const ExpressionActionsPtr & actions, const Settings & settings_);
void pushLevel(const NamesAndTypesList & input_columns);
size_t getColumnLevel(const std::string & name);
void addAction(const ExpressionAction & action);
ExpressionActionsPtr popLevel();
const Block & getSampleBlock() const;
};
/** Transforms an expression from a syntax tree into a sequence of actions to execute it.
*
@ -140,6 +167,7 @@ public:
/// Create Set-s that we can from IN section to use the index on them.
void makeSetsForIndex();
private:
ASTPtr ast;
ASTSelectQuery * select_query;
@ -271,8 +299,10 @@ private:
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;
struct ScopeStack;
void getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack);
bool isThereArrayJoin(const ASTPtr & ast);
void getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack,
ProjectionManipulatorPtr projection_manipulator);
void getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions);

View File

@ -0,0 +1,372 @@
#include <Common/Exception.h>
#include <Interpreters/ExpressionActions.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Interpreters/ProjectionManipulation.h>
#include <vector>
#include <string>
#include <memory>
#include <Common/typeid_cast.h>
namespace DB {
ProjectionManipulatorBase::~ProjectionManipulatorBase()
{}
DefaultProjectionManipulator::DefaultProjectionManipulator(ScopeStack & scopes)
: scopes(scopes)
{}
bool DefaultProjectionManipulator::tryToGetFromUpperProjection(const std::string & column_name)
{
return scopes.getSampleBlock().has(column_name);
}
std::string DefaultProjectionManipulator::getColumnName(const std::string & column_name) const
{
return column_name;
}
std::string DefaultProjectionManipulator::getProjectionExpression()
{
return "";
}
std::string DefaultProjectionManipulator::getProjectionSourceColumn() const {
return "";
}
ConditionalTree::Node::Node()
: projection_expression_string(),
parent_node(0),
is_root(false)
{}
size_t ConditionalTree::Node::getParentNode() const
{
if (is_root)
{
throw Exception("Failed to get parent projection node of node " + projection_expression_string,
ErrorCodes::CONDITIONAL_TREE_PARENT_NOT_FOUND);
}
else
{
return parent_node;
}
}
std::string ConditionalTree::getColumnNameByIndex(const std::string & col_name, const size_t node) const
{
std::string projection_name = nodes[node].projection_expression_string;
if (projection_name.empty())
{
return col_name;
}
else
{
return col_name + '<' + projection_name + '>';
}
}
std::string ConditionalTree::getColumnName(const std::string & col_name) const
{
return getColumnNameByIndex(col_name, current_node);
}
std::string ConditionalTree::getProjectionColumnName(const std::string & first_projection_expr,
const std::string & second_projection_expr) const
{
return std::string("P<") + first_projection_expr + "><" + second_projection_expr + ">";
}
std::string ConditionalTree::getProjectionColumnName(const size_t first_index, const size_t second_index) const
{
return getProjectionColumnName(
nodes[first_index].projection_expression_string,
nodes[second_index].projection_expression_string);
}
void ConditionalTree::buildProjectionCompositionRecursive(const std::vector<size_t> & path,
const size_t child_index,
const size_t parent_index)
{
std::string projection_name = getProjectionColumnName(path[parent_index], path[child_index]);
if (parent_index - child_index >= 2 && !scopes.getSampleBlock().has(projection_name))
{
size_t middle_index = (child_index + parent_index) / 2;
buildProjectionCompositionRecursive(path, child_index, middle_index);
buildProjectionCompositionRecursive(path, middle_index, parent_index);
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(
"__inner_build_projection_composition__", context);
scopes.addAction(ExpressionAction::applyFunction(
function_builder,
{
getProjectionColumnName(path[parent_index], path[middle_index]),
getProjectionColumnName(path[middle_index], path[child_index])
},
projection_name, getProjectionSourceColumn()));
}
}
void ConditionalTree::buildProjectionComposition(const size_t child_node, const size_t parent_node)
{
std::vector<size_t> path;
size_t node = child_node;
while (true)
{
path.push_back(node);
if (node == parent_node)
{
break;
}
node = nodes[node].getParentNode();
}
buildProjectionCompositionRecursive(path, 0, path.size() - 1);
}
std::string ConditionalTree::getProjectionSourceColumn(size_t node) const {
if (nodes[node].is_root)
{
return "";
}
else
{
return ConditionalTree::getProjectionColumnName(nodes[node].getParentNode(), node);
}
}
ConditionalTree::ConditionalTree(ScopeStack & scopes, const Context & context)
: current_node(0),
nodes(1),
scopes(scopes),
context(context),
projection_expression_index()
{
nodes[0].is_root = true;
}
void ConditionalTree::goToProjection(const std::string & field_name)
{
std::string current_projection_name = nodes[current_node].projection_expression_string;
std::string new_projection_name = current_projection_name.empty() ? field_name : current_projection_name + ";" + field_name;
std::string projection_column_name = getProjectionColumnName(current_projection_name, new_projection_name);
if (!scopes.getSampleBlock().has(projection_column_name))
{
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("one_or_zero", context);
scopes.addAction(ExpressionAction::applyFunction(function_builder, {getColumnName(field_name)}, projection_column_name,
getProjectionSourceColumn()));
nodes.emplace_back(Node());
nodes.back().projection_expression_string = new_projection_name;
nodes.back().parent_node = current_node;
current_node = nodes.size() - 1;
projection_expression_index[projection_column_name] = current_node;
} else {
current_node = projection_expression_index[projection_column_name];
}
}
std::string ConditionalTree::buildRestoreProjectionAndGetName(const size_t levels_up)
{
size_t target_node = current_node;
for (size_t i = 0; i < levels_up; ++i)
{
target_node = nodes[target_node].getParentNode();
}
buildProjectionComposition(current_node, target_node);
return getProjectionColumnName(target_node, current_node);
}
void ConditionalTree::restoreColumn(
const std::string & default_values_name,
const std::string & new_values_name,
const size_t levels_up,
const std::string & result_name
)
{
size_t target_node = current_node;
for (size_t i = 0; i < levels_up; ++i)
{
target_node = nodes[target_node].getParentNode();
}
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_restore_projection__",
context);
scopes.addAction(ExpressionAction::applyFunction(
function_builder,
{
getProjectionColumnName(target_node, current_node),
getColumnNameByIndex(default_values_name, current_node),
getColumnNameByIndex(new_values_name, current_node)
},
getColumnNameByIndex(result_name, target_node), getProjectionSourceColumn()));
}
void ConditionalTree::goUp(const size_t levels_up)
{
for (size_t i = 0; i < levels_up; ++i)
{
current_node = nodes[current_node].getParentNode();
}
}
bool ConditionalTree::tryToGetFromUpperProjection(const std::string & column_name)
{
size_t node = current_node;
while (true)
{
if (scopes.getSampleBlock().has(getColumnNameByIndex(column_name, node)))
{
if (node != current_node)
{
buildProjectionComposition(current_node, node);
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_project__",
context);
scopes.addAction(ExpressionAction::applyFunction(
function_builder,
{
getColumnNameByIndex(column_name, node),
getProjectionColumnName(node, current_node)
},
getColumnName(column_name), getProjectionSourceColumn(node)));
}
return true;
}
if (nodes[node].is_root)
{
break;
}
node = nodes[node].getParentNode();
}
return false;
}
std::string ConditionalTree::getProjectionExpression()
{
return nodes[current_node].projection_expression_string;
}
std::string ConditionalTree::getProjectionSourceColumn() const {
return getProjectionSourceColumn(current_node);
}
void DefaultProjectionAction::preArgumentAction()
{}
void DefaultProjectionAction::postArgumentAction(const std::string & /*argument_name*/)
{
}
void DefaultProjectionAction::preCalculation()
{}
bool DefaultProjectionAction::isCalculationRequired()
{
return true;
}
AndOperatorProjectionAction::AndOperatorProjectionAction(ScopeStack & scopes,
ProjectionManipulatorPtr projection_manipulator,
const std::string & expression_name,
const Context & context)
: scopes(scopes),
projection_manipulator(projection_manipulator),
previous_argument_name(),
projection_levels_count(0),
expression_name(expression_name),
context(context)
{}
std::string AndOperatorProjectionAction::getZerosColumnName()
{
return "__inner_zeroes_column__" + expression_name;
}
std::string AndOperatorProjectionAction::getFinalColumnName()
{
return "__inner_final_column__" + expression_name;
}
void AndOperatorProjectionAction::createZerosColumn(const std::string & restore_projection_name)
{
auto zeros_column_name = projection_manipulator->getColumnName(getZerosColumnName());
if (!scopes.getSampleBlock().has(zeros_column_name))
{
scopes.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
ColumnUInt8::create(0, 1), std::make_shared<DataTypeUInt8>(), zeros_column_name),
restore_projection_name, true));
}
}
void AndOperatorProjectionAction::preArgumentAction()
{
if (!previous_argument_name.empty())
{
// Before processing arguments starting from second to last
if (auto * conditional_tree = typeid_cast<ConditionalTree *>(projection_manipulator.get()))
{
conditional_tree->goToProjection(previous_argument_name);
}
else
{
throw Exception("Illegal projection manipulator used in AndOperatorProjectionAction", ErrorCodes::ILLEGAL_PROJECTION_MANIPULATOR);
}
++projection_levels_count;
}
}
void AndOperatorProjectionAction::postArgumentAction(const std::string & argument_name)
{
previous_argument_name = argument_name;
}
void AndOperatorProjectionAction::preCalculation()
{
if (auto * conditional_tree = typeid_cast<ConditionalTree *>(projection_manipulator.get()))
{
auto final_column = getFinalColumnName();
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("one_or_zero", context);
scopes.addAction(ExpressionAction::applyFunction(
function_builder,
{
projection_manipulator->getColumnName(previous_argument_name)
},
projection_manipulator->getColumnName(final_column),
projection_manipulator->getProjectionSourceColumn()));
std::string restore_projection_name = conditional_tree->buildRestoreProjectionAndGetName(projection_levels_count);
createZerosColumn(restore_projection_name);
conditional_tree->restoreColumn(getZerosColumnName(), final_column,
projection_levels_count, expression_name);
conditional_tree->goUp(projection_levels_count);
}
else
{
throw Exception("Illegal projection manipulator used in AndOperatorProjectionAction", ErrorCodes::ILLEGAL_PROJECTION_MANIPULATOR);
}
}
bool AndOperatorProjectionAction::isCalculationRequired()
{
return false;
}
ProjectionActionBase::~ProjectionActionBase()
{}
ProjectionActionPtr getProjectionAction(const std::string & node_name,
ScopeStack & scopes,
ProjectionManipulatorPtr projection_manipulator,
const std::string & expression_name,
const Context & context)
{
if (typeid_cast<ConditionalTree *>(projection_manipulator.get()) && node_name == "and")
{
return std::make_shared<AndOperatorProjectionAction>(scopes, projection_manipulator, expression_name, context);
}
else
{
return std::make_shared<DefaultProjectionAction>();
}
}
}

View File

@ -0,0 +1,249 @@
#pragma once
#include <string>
#include <Interpreters/ExpressionAnalyzer.h>
namespace DB {
class ExpressionAnalyzer;
struct ScopeStack;
namespace ErrorCodes {
extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND;
extern const int ILLEGAL_PROJECTION_MANIPULATOR;
}
/*
* This is a base class for the ConditionalTree. Look at the description of ConditionalTree.
*/
struct ProjectionManipulatorBase {
public:
virtual bool tryToGetFromUpperProjection(const std::string & column_name) = 0;
virtual std::string getColumnName(const std::string & col_name) const = 0;
virtual std::string getProjectionExpression() = 0;
virtual std::string getProjectionSourceColumn() const = 0;
virtual ~ProjectionManipulatorBase();
};
using ProjectionManipulatorPtr = std::shared_ptr<ProjectionManipulatorBase>;
/*
* This is the default ProjectionManipulator. It is needed for backwards compatibility.
* For the better understanding of what ProjectionManipulator does,
* look at the description of ConditionalTree.
*/
struct DefaultProjectionManipulator : public ProjectionManipulatorBase {
private:
ScopeStack & scopes;
public:
explicit DefaultProjectionManipulator(ScopeStack & scopes);
bool tryToGetFromUpperProjection(const std::string & column_name) final;
std::string getColumnName(const std::string & col_name) const final;
std::string getProjectionExpression() final;
std::string getProjectionSourceColumn() const final;
};
/*
* ConditionalTree is a projection manipulator. It is used in ExpressionAnalyzer::getActionsImpl.
* It is a helper class, which helps to build sequence of ExpressionAction instances -- actions, needed for
* computation of expression. It represents the current state of a projection layer. That is, if we have an expression
* f and g, we need to calculate f, afterwards we need to calculate g on the projection layer <f != 0>.
* This projection layer is stored in the ConditionalTree. Also, it stores the tree of all projection layers, which
* was seen before. If we have seen the projection layer <f != 0> and <f != 0 and g != 0>, conditional tree will put
* the second layer as a child to the first one.
*
* The description of what methods do:
* 1) getColumnName -- constructs the name of expression. which contains the information of the projection layer.
* It is needed to make computed column name unique. That is, if we have an expression g and conditional layer
* <f != 0>, it forms the name g<f != 0>
*
* 2) goToProjection -- accepts field name f and builds child projection layer with the additional condition
* <f>. For instance, if we are on the projection layer a != 0 and the function accepts the expression b != 0,
* it will build a projection layer <a != 0 and b != 0>, and remember that this layer is a child to a previous one.
* Moreover, the function will store the actions to build projection between this two layers in the corresponding
* ScopeStack
*
* 3) restoreColumn(default_values_name, new_values_name, levels, result_name) -- stores action to restore calculated
* 'new_values_name' column, to insert its values to the projection layer, which is 'levels' number of levels higher.
*
* 4) goUp -- goes several levels up in the conditional tree, raises the exception if we hit the root of the tree and
* there are still remained some levels up to go.
*
* 5) tryToGetFromUpperProjection -- goes up to the root projection level and checks whether the expression is
* already calculated somewhere in the higher projection level. If it is, we may just project it to the current
* layer to have it computed in the current layer. In this case, the function stores all actions needed to compute
* the projection: computes composition of projections and uses it to project the column. In the other case, if
* the column is not computed on the higher level, the function returns false. It is used in getActinosImpl to
* understand whether we need to scan the expression deeply, or can it be easily computed just with the projection
* from one of the higher projection layers.
*/
struct ConditionalTree : public ProjectionManipulatorBase {
private:
struct Node {
Node();
size_t getParentNode() const;
std::string projection_expression_string;
size_t parent_node;
bool is_root;
};
size_t current_node;
std::vector<Node> nodes;
ScopeStack & scopes;
const Context & context;
std::unordered_map<std::string, size_t> projection_expression_index;
private:
std::string getColumnNameByIndex(const std::string & col_name, size_t node) const;
std::string getProjectionColumnName(const std::string & first_projection_expr,
const std::string & second_projection_expr) const;
std::string getProjectionColumnName(size_t first_index, size_t second_index) const;
void buildProjectionCompositionRecursive(const std::vector<size_t> & path,
size_t child_index,
size_t parent_index);
void buildProjectionComposition(size_t child_node, size_t parent_node);
std::string getProjectionSourceColumn(size_t node) const;
public:
ConditionalTree(ScopeStack & scopes, const Context & context);
std::string getColumnName(const std::string & col_name) const final;
void goToProjection(const std::string & field_name);
std::string buildRestoreProjectionAndGetName(size_t levels_up);
void restoreColumn(
const std::string & default_values_name,
const std::string & new_values_name,
size_t levels_up,
const std::string & result_name
);
void goUp(size_t levels_up);
bool tryToGetFromUpperProjection(const std::string & column_name) final;
std::string getProjectionExpression() final;
std::string getProjectionSourceColumn() const final;
};
using ConditionalTreePtr = std::shared_ptr<ConditionalTree>;
/*
* ProjectionAction describes in what way should some specific function use the projection manipulator.
* This class has two inherited classes: DefaultProjectionAction, which does nothing, and AndOperatorProjectionAction,
* which represents how function "and" uses projection manipulator.
*/
class ProjectionActionBase {
public:
/*
* What to do before scanning the function argument (each of it)
*/
virtual void preArgumentAction() = 0;
/*
* What to do after scanning each argument
*/
virtual void postArgumentAction(const std::string & argument_name) = 0;
/*
* What to do after scanning all the arguments, before the computation
*/
virtual void preCalculation() = 0;
/*
* Should default computation procedure be run or not
*/
virtual bool isCalculationRequired() = 0;
virtual ~ProjectionActionBase();
};
using ProjectionActionPtr = std::shared_ptr<ProjectionActionBase>;
class DefaultProjectionAction : public ProjectionActionBase {
public:
void preArgumentAction() final;
void postArgumentAction(const std::string & argument_name) final;
void preCalculation() final;
bool isCalculationRequired() final;
};
/*
* This is a specification of ProjectionAction specifically for the 'and' operation
*/
class AndOperatorProjectionAction : public ProjectionActionBase {
private:
ScopeStack & scopes;
ProjectionManipulatorPtr projection_manipulator;
std::string previous_argument_name;
size_t projection_levels_count;
std::string expression_name;
const Context & context;
std::string getZerosColumnName();
std::string getFinalColumnName();
void createZerosColumn(const std::string & restore_projection_name);
public:
AndOperatorProjectionAction(ScopeStack & scopes,
ProjectionManipulatorPtr projection_manipulator,
const std::string & expression_name,
const Context& context);
/*
* Before scanning each argument, we should go to the next projection layer. For example, if the expression is
* f and g and h, then before computing g we should project to <f != 0> and before computing h we should project to
* <f != 0 and g != 0>
*/
void preArgumentAction() final;
/*
* Stores the previous argument name
*/
void postArgumentAction(const std::string & argument_name) final;
/*
* Restores the result column to the uppermost projection level. For example, if the expression is f and g and h,
* we should restore h<f,g> to the main projection layer
*/
void preCalculation() final;
/*
* After what is done in preCalculation, we do not need to run default calculation of 'and' operator. So, the
* function returns false.
*/
bool isCalculationRequired() final;
};
/*
* This function accepts the operator name and returns its projection action. For example, for 'and' operator,
* it returns the pointer to AndOperatorProjectionAction.
*/
ProjectionActionPtr getProjectionAction(const std::string & node_name,
ScopeStack & scopes,
ProjectionManipulatorPtr projection_manipulator,
const std::string & expression_name,
const Context & context);
}

View File

@ -247,7 +247,9 @@ struct Settings
M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.") \
M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.") \
M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.")\
M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.")
M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.") \
M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};

View File

@ -26,6 +26,7 @@ namespace ErrorCodes
extern const int UNKNOWN_COMPRESSION_METHOD;
extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE;
extern const int UNKNOWN_GLOBAL_SUBQUERIES_METHOD;
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
}
@ -706,4 +707,58 @@ struct SettingString
}
};
struct SettingChar
{
private:
void checkStringIsACharacter(const String & x) const
{
if (x.size() != 1)
throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH);
}
public:
char value;
bool changed = false;
SettingChar(char x = '\0') : value(x) {}
operator char() const { return value; }
SettingChar & operator= (char x) { set(x); return *this; }
String toString() const
{
return String(1, value);
}
void set(char x) {
value = x;
changed = true;
}
void set(const String & x)
{
checkStringIsACharacter(x);
value = x[0];
changed = true;
}
void set(const Field & x)
{
const String & s = safeGet<const String &>(x);
set(s);
}
void set(ReadBuffer & buf)
{
String x;
readBinary(x, buf);
checkStringIsACharacter(x);
set(x);
}
void write(WriteBuffer & buf) const
{
writeBinary(toString(), buf);
}
};
}

View File

@ -32,7 +32,7 @@ ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type
ColumnsWithTypeAndName arguments{ temporary_block.getByPosition(0), temporary_block.getByPosition(1) };
auto func_cast = func_builder_cast->build(arguments);
func_cast->execute(temporary_block, {0, 1}, 2);
func_cast->execute(temporary_block, {0, 1}, 2, arg.column->size());
return temporary_block.getByPosition(2).column;
}

View File

@ -362,7 +362,7 @@ static void applyFunction(
{ nullptr, res_type, "y" }
};
func->execute(block, {0}, 1);
func->execute(block, {0}, 1, 1);
block.safeGetByPosition(1).column->get(0, res_value);
}

View File

@ -1008,7 +1008,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name
/// This is temporary name for expression. TODO Invent the name more safely.
const String new_type_name_column = '#' + new_type_name + "_column";
out_expression->add(ExpressionAction::addColumn(
{ DataTypeString().createColumnConst(1, new_type_name), std::make_shared<DataTypeString>(), new_type_name_column }));
{ DataTypeString().createColumnConst(1, new_type_name), std::make_shared<DataTypeString>(), new_type_name_column }, "", false));
const auto & function = FunctionFactory::instance().get("CAST", context);
out_expression->add(ExpressionAction::applyFunction(

View File

@ -0,0 +1,30 @@
Hello, world 123 2016-01-01
Hello, "world" 456 2016-01-02
Hello "world" 789 2016-01-03
Hello\n world 100 2016-01-04
Hello, world 123 2016-01-01
Hello, "world" 456 2016-01-02
Hello "world" 789 2016-01-03
Hello\n world 100 2016-01-04
"Hello, world";123;"2016-01-01"
"Hello, ""world""";456;"2016-01-02"
"Hello ""world""";789;"2016-01-03"
"Hello
world";100;"2016-01-04"
"Hello, world"/123/"2016-01-01"
"Hello, ""world"""/456/"2016-01-02"
"Hello ""world"""/789/"2016-01-03"
"Hello
world"/100/"2016-01-04"
abc,def hello
hello world
hello "world" abc,def
"abc,def";"hello"
"hello";"world"
"hello ""world""";"abc,def"
"abc,def","hello"
"hello","world"
"hello ""world""","abc,def"
"abc,def"/"hello"
"hello"/"world"
"hello ""world"""/"abc,def"

View File

@ -0,0 +1,52 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
echo '"Hello, world"| 123| "2016-01-01"
"Hello, ""world"""| "456"| 2016-01-02|
Hello "world"| 789 |2016-01-03
"Hello
world"| 100| 2016-01-04|' | $CLICKHOUSE_CLIENT --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
echo '"Hello, world"; 123; "2016-01-01"
"Hello, ""world"""; "456"; 2016-01-02;
Hello "world"; 789 ;2016-01-03
"Hello
world"; 100; 2016-01-04;' | $CLICKHOUSE_CLIENT --multiquery --query="SET format_csv_delimiter=';'; INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --format_csv_delimiter=";" --query="SELECT * FROM test.csv ORDER BY d FORMAT CSV";
$CLICKHOUSE_CLIENT --format_csv_delimiter="/" --query="SELECT * FROM test.csv ORDER BY d FORMAT CSV";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s1 String, s2 String) ENGINE = Memory";
echo 'abc,def;hello;
hello; world;
"hello ""world""";abc,def;' | $CLICKHOUSE_CLIENT --multiquery --query="SET format_csv_delimiter=';'; INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s1 String, s2 String) ENGINE = Memory";
echo '"s1";"s2"
abc,def;hello;
hello; world;
"hello ""world""";abc,def;' | $CLICKHOUSE_CLIENT --multiquery --query="SET format_csv_delimiter=';'; INSERT INTO test.csv FORMAT CSVWithNames";
$CLICKHOUSE_CLIENT --format_csv_delimiter=";" --query="SELECT * FROM test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --format_csv_delimiter="," --query="SELECT * FROM test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --format_csv_delimiter="/" --query="SELECT * FROM test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";

View File

@ -0,0 +1,4 @@
0 0 0 0 0 0 0
0 5 0 0 0 0 0
6 3 1 1 1 1 0
7 0 0 0 0 0 0

View File

@ -0,0 +1,23 @@
USE test;
DROP TABLE IF EXISTS test;
CREATE TABLE test (d Date DEFAULT '2000-01-01', x UInt64, y UInt64) ENGINE = MergeTree(d, x, 1);
INSERT INTO test(x,y) VALUES (6, 3);
INSERT INTO test(x,y) VALUES (0, 5);
INSERT INTO test(x,y) VALUES (7, 0);
INSERT INTO test(x,y) VALUES (0, 0);
SET enable_conditional_computation=1;
SELECT
x,
y,
x and y,
y and x,
x and 1 and x and y,
x and modulo(y, x),
y and modulo(x,y)
FROM
test
ORDER BY
x, y
;
SET enable_conditional_computation=0;

View File

@ -2,9 +2,11 @@
Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)).
When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by commas. Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by a delimiter&ast;. Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to a comma or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) are all supported.
&ast;By default — `,`. See a [format_csv_delimiter](/docs/en/operations/settings/settings/#format_csv_delimiter) setting for additional info.
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to a delimiter or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) are all supported.
The CSV format supports the output of totals and extremes the same way as `TabSeparated`.

View File

@ -4,8 +4,7 @@
Web interface for ClickHouse in the [Tabix](https://github.com/tabixio/tabix) project.
Main features:
### Features:
- Works with ClickHouse directly from the browser, without the need to install additional software.
- Query editor with syntax highlighting.
- Auto-completion of commands.
@ -14,3 +13,17 @@ Main features:
[Tabix documentation](https://tabix.io/doc/).
## HouseOps
[HouseOps](https://github.com/HouseOps/HouseOps) is a unique Desktop ClickHouse Ops UI / IDE for OSX, Linux and Windows.
### Features:
- Query builder;
- Database manangement (soon);
- Users manangement (soon);
- Real-Time Data Analytics (soon);
- Cluster/Infra monitoring (soon);
- Cluster manangement (soon);
- Kafka and Replicated tables monitoring (soon);
- And a lot of others features (soon) for you take a beautiful implementation of ClickHouse.

View File

@ -338,3 +338,7 @@ It works for JSONEachRow and TSKV formats.
## output_format_json_quote_64bit_integers
If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes.
## format_csv_delimiter
The character to be considered as a delimiter in CSV data. By default, `,`.

View File

@ -2,8 +2,10 @@
Формат comma separated values ([RFC](https://tools.ietf.org/html/rfc4180)).
При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются запятыми. Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж).
При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются символом-разделителем&ast;. Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж).
При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. В том числе, строки могут быть расположены без кавычек - тогда они парсятся до запятой или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты.
&ast;По умолчанию — `,`. См. настройку [format_csv_delimiter](/docs/ru/operations/settings/settings/#format_csv_delimiter) для дополнительной информации.
При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. В том числе, строки могут быть расположены без кавычек - тогда они парсятся до символа-разделителя или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты.
Формат CSV поддерживает вывод totals и extremes аналогично `TabSeparated`.

View File

@ -12,3 +12,7 @@
- Цветовые схемы на выбор.
[Документация Tabix](https://tabix.io/doc/).
## HouseOps
[HouseOps](https://github.com/HouseOps/HouseOps) is a unique Desktop ClickHouse Ops UI / IDE for OSX, Linux and Windows.

View File

@ -335,3 +335,7 @@ ClickHouse применяет настройку в том случае, ког
## output_format_json_quote_64bit_integers
Если значение истинно, то при использовании JSON\* форматов UInt64 и Int64 числа выводятся в кавычках (из соображений совместимости с большинством реализаций JavaScript), иначе - без кавычек.
## format_csv_delimiter
Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`.

View File

@ -90,11 +90,6 @@
<div class="clear"></div>
</div>
</div>
<div id="announcement" class="colored-block">
<div class="page">
ClickHouse Meetup in <a id="announcement-link" href="https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/248898966/" rel="external nofollow" target="_blank">Sunnyvale</a> &amp; <a id="announcement-link" href="https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/249162518/" rel="external nofollow" target="_blank">San Francisco</a>, April 23-27</a>
</div>
</div>
<div class="page">
<h2 id="slogan">ClickHouse. Just makes you think faster.</h2>