ClickHouse/src/Storages/MergeTree/MergeTreeIndexSet.cpp

496 lines
15 KiB
C++
Raw Normal View History

2019-06-19 15:30:48 +00:00
#include <Storages/MergeTree/MergeTreeIndexSet.h>
2019-01-26 13:12:13 +00:00
2019-01-26 15:45:38 +00:00
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
2019-01-27 21:08:14 +00:00
#include <Parsers/ASTIdentifier.h>
2019-01-27 20:24:33 +00:00
#include <Parsers/ASTFunction.h>
2019-01-27 21:08:14 +00:00
#include <Parsers/ASTLiteral.h>
2019-01-26 15:45:38 +00:00
2019-02-06 11:08:04 +00:00
2019-01-26 13:12:13 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int LOGICAL_ERROR;
2019-01-26 13:12:13 +00:00
extern const int INCORRECT_QUERY;
}
2019-02-05 13:52:45 +00:00
/// 0b11 -- can be true and false at the same time
static const Field UNKNOWN_FIELD(3u);
2019-02-05 13:52:45 +00:00
2020-05-27 18:38:34 +00:00
MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet(
const String & index_name_,
const Block & index_sample_block_,
size_t max_rows_)
: index_name(index_name_)
, max_rows(max_rows_)
, index_sample_block(index_sample_block_)
, block(index_sample_block)
{
}
2019-03-08 19:52:21 +00:00
2019-06-19 15:30:48 +00:00
MergeTreeIndexGranuleSet::MergeTreeIndexGranuleSet(
2020-05-27 18:38:34 +00:00
const String & index_name_,
const Block & index_sample_block_,
size_t max_rows_,
MutableColumns && mutable_columns_)
: index_name(index_name_)
, max_rows(max_rows_)
, index_sample_block(index_sample_block_)
, block(index_sample_block.cloneWithColumns(std::move(mutable_columns_)))
{
}
2019-01-26 13:12:13 +00:00
2019-06-19 15:30:48 +00:00
void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const
2019-01-26 13:12:13 +00:00
{
if (empty())
throw Exception(
2020-05-27 18:38:34 +00:00
"Attempt to write empty set index " + backQuote(index_name), ErrorCodes::LOGICAL_ERROR);
2019-01-26 13:12:13 +00:00
2019-01-27 18:02:15 +00:00
const auto & size_type = DataTypePtr(std::make_shared<DataTypeUInt64>());
2019-01-29 17:26:45 +00:00
2020-05-27 18:38:34 +00:00
if (max_rows != 0 && size() > max_rows)
2019-01-29 17:26:45 +00:00
{
size_type->serializeBinary(0, ostr);
return;
}
2019-01-27 18:02:15 +00:00
size_type->serializeBinary(size(), ostr);
2020-05-27 18:38:34 +00:00
for (size_t i = 0; i < index_sample_block.columns(); ++i)
2019-01-26 13:12:13 +00:00
{
2020-05-27 18:38:34 +00:00
const auto & type = index_sample_block.getByPosition(i).type;
IDataType::SerializeBinaryBulkSettings settings;
settings.getter = [&ostr](IDataType::SubstreamPath) -> WriteBuffer * { return &ostr; };
settings.position_independent_encoding = false;
settings.low_cardinality_max_dictionary_size = 0;
IDataType::SerializeBinaryBulkStatePtr state;
type->serializeBinaryBulkStatePrefix(settings, state);
2019-03-08 19:52:21 +00:00
type->serializeBinaryBulkWithMultipleStreams(*block.getByPosition(i).column, 0, size(), settings, state);
type->serializeBinaryBulkStateSuffix(settings, state);
2019-01-26 13:12:13 +00:00
}
}
2019-06-19 15:30:48 +00:00
void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr)
2019-01-26 13:12:13 +00:00
{
2019-03-08 19:52:21 +00:00
block.clear();
2019-01-27 18:02:15 +00:00
Field field_rows;
const auto & size_type = DataTypePtr(std::make_shared<DataTypeUInt64>());
size_type->deserializeBinary(field_rows, istr);
size_t rows_to_read = field_rows.get<size_t>();
if (rows_to_read == 0)
return;
2020-05-27 18:38:34 +00:00
for (size_t i = 0; i < index_sample_block.columns(); ++i)
2019-01-26 13:12:13 +00:00
{
2020-05-27 18:38:34 +00:00
const auto & column = index_sample_block.getByPosition(i);
const auto & type = column.type;
2020-11-10 17:32:00 +00:00
ColumnPtr new_column = type->createColumn();
IDataType::DeserializeBinaryBulkSettings settings;
settings.getter = [&](IDataType::SubstreamPath) -> ReadBuffer * { return &istr; };
settings.position_independent_encoding = false;
IDataType::DeserializeBinaryBulkStatePtr state;
type->deserializeBinaryBulkStatePrefix(settings, state);
2020-11-10 17:32:00 +00:00
type->deserializeBinaryBulkWithMultipleStreams(new_column, rows_to_read, settings, state);
2019-01-26 13:12:13 +00:00
2020-11-10 17:32:00 +00:00
block.insert(ColumnWithTypeAndName(new_column, type, column.name));
2019-01-26 13:12:13 +00:00
}
2019-03-08 19:52:21 +00:00
}
2019-01-27 18:02:15 +00:00
2020-05-27 18:38:34 +00:00
MergeTreeIndexAggregatorSet::MergeTreeIndexAggregatorSet(const String & index_name_, const Block & index_sample_block_, size_t max_rows_)
: index_name(index_name_)
, max_rows(max_rows_)
, index_sample_block(index_sample_block_)
, columns(index_sample_block_.cloneEmptyColumns())
2019-03-08 19:52:21 +00:00
{
2019-03-09 14:03:41 +00:00
ColumnRawPtrs column_ptrs;
2020-05-27 18:38:34 +00:00
column_ptrs.reserve(index_sample_block.columns());
2019-03-09 14:03:41 +00:00
Columns materialized_columns;
2020-05-27 18:38:34 +00:00
for (const auto & column : index_sample_block.getColumns())
2019-03-09 14:03:41 +00:00
{
materialized_columns.emplace_back(column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality());
column_ptrs.emplace_back(materialized_columns.back().get());
}
data.init(ClearableSetVariants::chooseMethod(column_ptrs, key_sizes));
2020-05-27 18:38:34 +00:00
columns = index_sample_block.cloneEmptyColumns();
2019-01-26 13:12:13 +00:00
}
2019-06-19 15:30:48 +00:00
void MergeTreeIndexAggregatorSet::update(const Block & block, size_t * pos, size_t limit)
2019-01-26 13:12:13 +00:00
{
2019-03-09 14:03:41 +00:00
if (*pos >= block.rows())
2019-02-06 07:49:18 +00:00
throw Exception(
"The provided position is not less than the number of block rows. Position: "
2019-03-09 14:03:41 +00:00
+ toString(*pos) + ", Block rows: " + toString(block.rows()) + ".", ErrorCodes::LOGICAL_ERROR);
2019-02-06 07:49:18 +00:00
2019-03-09 14:03:41 +00:00
size_t rows_read = std::min(limit, block.rows() - *pos);
2019-01-26 13:12:13 +00:00
2020-05-27 18:38:34 +00:00
if (max_rows && size() > max_rows)
2019-01-26 13:12:13 +00:00
{
2019-01-27 18:02:15 +00:00
*pos += rows_read;
return;
2019-01-26 13:12:13 +00:00
}
2019-03-09 14:03:41 +00:00
ColumnRawPtrs index_column_ptrs;
2020-05-27 18:38:34 +00:00
index_column_ptrs.reserve(index_sample_block.columns());
2019-03-09 14:03:41 +00:00
Columns materialized_columns;
2020-05-27 18:38:34 +00:00
const Names index_columns = index_sample_block.getNames();
for (const auto & column_name : index_columns)
2019-01-26 13:12:13 +00:00
{
2019-03-09 14:03:41 +00:00
materialized_columns.emplace_back(
block.getByName(column_name).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality());
index_column_ptrs.emplace_back(materialized_columns.back().get());
2019-01-26 13:12:13 +00:00
}
2019-01-27 18:02:15 +00:00
2019-03-09 14:03:41 +00:00
IColumn::Filter filter(block.rows(), 0);
bool has_new_data = false;
switch (data.type)
{
case ClearableSetVariants::Type::EMPTY:
break;
#define M(NAME) \
case ClearableSetVariants::Type::NAME: \
has_new_data = buildFilter(*data.NAME, index_column_ptrs, filter, *pos, rows_read, data); \
break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
2019-03-09 18:17:29 +00:00
if (has_new_data)
2019-03-09 14:03:41 +00:00
{
2019-03-09 18:17:29 +00:00
for (size_t i = 0; i < columns.size(); ++i)
{
2020-05-27 18:38:34 +00:00
auto filtered_column = block.getByName(index_columns[i]).column->filter(filter, block.rows());
2019-03-09 18:17:29 +00:00
columns[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size());
}
2019-03-09 14:03:41 +00:00
}
2019-01-27 18:02:15 +00:00
*pos += rows_read;
2019-01-27 18:23:08 +00:00
}
2019-03-09 14:03:41 +00:00
template <typename Method>
2019-06-19 15:30:48 +00:00
bool MergeTreeIndexAggregatorSet::buildFilter(
2019-03-09 14:03:41 +00:00
Method & method,
2019-03-09 15:26:52 +00:00
const ColumnRawPtrs & column_ptrs,
2019-03-09 14:03:41 +00:00
IColumn::Filter & filter,
size_t pos,
size_t limit,
ClearableSetVariants & variants) const
2019-01-27 18:23:08 +00:00
{
2019-03-09 14:03:41 +00:00
/// Like DistinctSortedBlockInputStream.
2019-03-09 15:26:52 +00:00
typename Method::State state(column_ptrs, key_sizes, nullptr);
2019-03-09 14:03:41 +00:00
bool has_new_data = false;
for (size_t i = 0; i < limit; ++i)
{
auto emplace_result = state.emplaceKey(method.data, pos + i, variants.string_pool);
if (emplace_result.isInserted())
has_new_data = true;
/// Emit the record if there is no such key in the current set yet.
/// Skip it otherwise.
filter[pos + i] = emplace_result.isInserted();
}
return has_new_data;
}
2019-06-19 15:30:48 +00:00
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorSet::getGranuleAndReset()
2019-01-27 18:23:08 +00:00
{
2020-05-27 18:38:34 +00:00
auto granule = std::make_shared<MergeTreeIndexGranuleSet>(index_name, index_sample_block, max_rows, std::move(columns));
2019-03-09 14:03:41 +00:00
switch (data.type)
{
case ClearableSetVariants::Type::EMPTY:
break;
#define M(NAME) \
case ClearableSetVariants::Type::NAME: \
data.NAME->data.clear(); \
break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
2019-03-08 19:52:21 +00:00
2020-05-27 18:38:34 +00:00
columns = index_sample_block.cloneEmptyColumns();
2019-03-08 19:52:21 +00:00
return granule;
2019-01-27 18:23:08 +00:00
}
2019-01-26 15:45:38 +00:00
2019-06-19 15:30:48 +00:00
MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
2020-05-27 18:38:34 +00:00
const String & index_name_,
const Block & index_sample_block_,
size_t max_rows_,
const SelectQueryInfo & query,
const Context & context)
: index_name(index_name_)
, max_rows(max_rows_)
, index_sample_block(index_sample_block_)
2019-01-27 20:24:33 +00:00
{
2020-05-27 18:38:34 +00:00
for (const auto & name : index_sample_block.getNames())
2019-01-27 20:24:33 +00:00
if (!key_columns.count(name))
2019-01-29 17:26:45 +00:00
key_columns.insert(name);
2019-01-27 20:24:33 +00:00
const auto & select = query.query->as<ASTSelectQuery &>();
2019-01-27 20:24:33 +00:00
if (select.where() && select.prewhere())
2019-02-05 18:34:10 +00:00
expression_ast = makeASTFunction(
2019-01-27 20:24:33 +00:00
"and",
select.where()->clone(),
select.prewhere()->clone());
else if (select.where())
expression_ast = select.where()->clone();
else if (select.prewhere())
expression_ast = select.prewhere()->clone();
2019-01-27 20:24:33 +00:00
2019-02-05 18:34:10 +00:00
useless = checkASTUseless(expression_ast);
2019-01-29 16:53:44 +00:00
/// Do not proceed if index is useless for this query.
2019-02-05 13:52:45 +00:00
if (useless)
2019-01-29 16:53:44 +00:00
return;
2019-08-04 13:03:38 +00:00
/// Replace logical functions with bit functions.
2020-04-03 15:14:31 +00:00
/// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h).
2019-01-28 08:15:13 +00:00
traverseAST(expression_ast);
auto syntax_analyzer_result = TreeRewriter(context).analyze(
2020-05-27 18:38:34 +00:00
expression_ast, index_sample_block.getNamesAndTypesList());
2019-01-29 13:13:18 +00:00
actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true);
2019-01-27 20:24:33 +00:00
}
2019-01-26 15:45:38 +00:00
2019-06-19 15:30:48 +00:00
bool MergeTreeIndexConditionSet::alwaysUnknownOrTrue() const
2019-01-26 15:45:38 +00:00
{
2019-01-29 16:53:44 +00:00
return useless;
2019-01-26 15:45:38 +00:00
}
2019-01-26 13:12:13 +00:00
2019-06-19 15:30:48 +00:00
bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
2019-01-26 15:45:38 +00:00
{
if (useless)
return true;
2019-06-19 15:30:48 +00:00
auto granule = std::dynamic_pointer_cast<MergeTreeIndexGranuleSet>(idx_granule);
2019-01-26 15:45:38 +00:00
if (!granule)
throw Exception(
2019-02-13 19:26:24 +00:00
"Set index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
2019-01-26 15:45:38 +00:00
2020-05-27 18:38:34 +00:00
if (useless || granule->empty() || (max_rows != 0 && granule->size() > max_rows))
2019-01-28 08:15:13 +00:00
return true;
2019-03-08 19:52:21 +00:00
Block result = granule->block;
2019-01-28 08:15:13 +00:00
actions->execute(result);
2020-08-16 07:16:37 +00:00
auto column
= result.getByName(expression_ast->getColumnName()).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality();
2020-04-22 06:34:20 +00:00
const auto * col_uint8 = typeid_cast<const ColumnUInt8 *>(column.get());
const NullMap * null_map = nullptr;
2020-04-22 06:34:20 +00:00
if (const auto * col_nullable = checkAndGetColumn<ColumnNullable>(*column))
{
col_uint8 = typeid_cast<const ColumnUInt8 *>(&col_nullable->getNestedColumn());
null_map = &col_nullable->getNullMapData();
}
if (!col_uint8)
throw Exception("ColumnUInt8 expected as Set index condition result.", ErrorCodes::LOGICAL_ERROR);
2020-04-22 06:34:20 +00:00
const auto & condition = col_uint8->getData();
2019-01-28 08:15:13 +00:00
for (size_t i = 0; i < column->size(); ++i)
if ((!null_map || (*null_map)[i] == 0) && condition[i] & 1)
2019-01-28 08:15:13 +00:00
return true;
return false;
2019-01-26 15:45:38 +00:00
}
2019-06-19 15:30:48 +00:00
void MergeTreeIndexConditionSet::traverseAST(ASTPtr & node) const
2019-01-27 20:24:33 +00:00
{
2019-01-29 18:40:10 +00:00
if (operatorFromAST(node))
{
2019-03-11 13:22:51 +00:00
auto & args = node->as<ASTFunction>()->arguments->children;
2019-01-27 20:24:33 +00:00
2019-01-29 17:09:17 +00:00
for (auto & arg : args)
traverseAST(arg);
2019-01-28 07:24:32 +00:00
return;
2019-01-27 20:24:33 +00:00
}
2019-08-01 19:18:36 +00:00
if (atomFromAST(node))
2019-08-02 19:21:55 +00:00
{
if (node->as<ASTIdentifier>() || node->as<ASTFunction>())
node = makeASTFunction("__bitWrapperFunc", node);
}
2019-08-01 19:18:36 +00:00
else
2019-02-05 13:52:45 +00:00
node = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);
2019-01-27 20:24:33 +00:00
}
2019-06-19 15:30:48 +00:00
bool MergeTreeIndexConditionSet::atomFromAST(ASTPtr & node) const
2019-01-27 20:24:33 +00:00
{
2019-01-27 21:08:14 +00:00
/// Function, literal or column
2019-03-11 13:22:51 +00:00
if (node->as<ASTLiteral>())
2019-01-27 21:08:14 +00:00
return true;
2019-03-11 13:22:51 +00:00
if (const auto * identifier = node->as<ASTIdentifier>())
2019-01-28 07:05:24 +00:00
return key_columns.count(identifier->getColumnName()) != 0;
2019-01-27 21:08:14 +00:00
2019-03-11 13:22:51 +00:00
if (auto * func = node->as<ASTFunction>())
2019-01-29 13:13:18 +00:00
{
2019-01-28 07:05:24 +00:00
if (key_columns.count(func->getColumnName()))
2019-01-29 13:13:18 +00:00
{
/// Function is already calculated.
node = std::make_shared<ASTIdentifier>(func->getColumnName());
2019-01-27 21:08:14 +00:00
return true;
2019-01-29 13:13:18 +00:00
}
2019-01-27 21:08:14 +00:00
auto & args = func->arguments->children;
2019-01-27 21:08:14 +00:00
2019-01-29 17:09:17 +00:00
for (auto & arg : args)
if (!atomFromAST(arg))
2019-01-27 20:24:33 +00:00
return false;
return true;
}
return false;
}
2020-03-18 00:57:00 +00:00
bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node)
2019-01-27 20:24:33 +00:00
{
/// Functions AND, OR, NOT. Replace with bit*.
2019-03-11 13:22:51 +00:00
auto * func = node->as<ASTFunction>();
2019-01-28 07:24:32 +00:00
if (!func)
return false;
auto & args = func->arguments->children;
2019-01-27 20:24:33 +00:00
if (func->name == "not")
{
if (args.size() != 1)
return false;
2019-01-28 07:24:32 +00:00
2019-01-30 19:40:01 +00:00
func->name = "__bitSwapLastTwo";
2019-01-27 20:24:33 +00:00
}
else if (func->name == "and")
2019-02-05 18:34:10 +00:00
{
auto last_arg = args.back();
args.pop_back();
ASTPtr new_func;
if (args.size() > 1)
new_func = makeASTFunction(
2019-08-02 19:21:55 +00:00
"__bitBoolMaskAnd",
2019-02-05 18:34:10 +00:00
node,
last_arg);
else
new_func = makeASTFunction(
2019-08-02 19:21:55 +00:00
"__bitBoolMaskAnd",
2019-02-05 18:34:10 +00:00
args.back(),
last_arg);
node = new_func;
}
2019-01-27 20:24:33 +00:00
else if (func->name == "or")
2019-02-05 18:34:10 +00:00
{
auto last_arg = args.back();
args.pop_back();
ASTPtr new_func;
if (args.size() > 1)
new_func = makeASTFunction(
2019-08-02 19:21:55 +00:00
"__bitBoolMaskOr",
2019-02-05 18:34:10 +00:00
node,
last_arg);
else
new_func = makeASTFunction(
2019-08-02 19:21:55 +00:00
"__bitBoolMaskOr",
2019-02-05 18:34:10 +00:00
args.back(),
last_arg);
node = new_func;
}
2019-01-27 20:24:33 +00:00
else
return false;
return true;
}
bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr & node, bool atomic) const
2019-01-29 16:53:44 +00:00
{
if (!node)
return true;
2019-03-11 13:22:51 +00:00
if (const auto * func = node->as<ASTFunction>())
2019-01-29 16:53:44 +00:00
{
if (key_columns.count(func->getColumnName()))
return false;
const ASTs & args = func->arguments->children;
2019-01-29 16:53:44 +00:00
if (func->name == "and")
2019-02-05 13:52:45 +00:00
return checkASTUseless(args[0], atomic) && checkASTUseless(args[1], atomic);
2019-01-29 16:53:44 +00:00
else if (func->name == "or")
2019-02-05 13:52:45 +00:00
return checkASTUseless(args[0], atomic) || checkASTUseless(args[1], atomic);
2019-01-29 16:53:44 +00:00
else if (func->name == "not")
2019-02-05 13:52:45 +00:00
return checkASTUseless(args[0], atomic);
2019-01-29 16:53:44 +00:00
else
return std::any_of(args.begin(), args.end(),
[this](const auto & arg) { return checkASTUseless(arg, true); });
2019-01-29 16:53:44 +00:00
}
2019-03-11 13:22:51 +00:00
else if (const auto * literal = node->as<ASTLiteral>())
2019-01-29 16:53:44 +00:00
return !atomic && literal->value.get<bool>();
2019-03-11 13:22:51 +00:00
else if (const auto * identifier = node->as<ASTIdentifier>())
return key_columns.find(identifier->getColumnName()) == std::end(key_columns);
2019-01-29 16:53:44 +00:00
else
return true;
}
2019-01-26 15:45:38 +00:00
2019-06-19 15:30:48 +00:00
MergeTreeIndexGranulePtr MergeTreeIndexSet::createIndexGranule() const
2019-01-26 15:45:38 +00:00
{
2020-05-27 18:38:34 +00:00
return std::make_shared<MergeTreeIndexGranuleSet>(index.name, index.sample_block, max_rows);
2019-01-26 15:45:38 +00:00
}
2019-01-26 13:12:13 +00:00
2019-06-19 15:30:48 +00:00
MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator() const
2019-03-08 19:52:21 +00:00
{
2020-05-27 18:38:34 +00:00
return std::make_shared<MergeTreeIndexAggregatorSet>(index.name, index.sample_block, max_rows);
2019-03-08 19:52:21 +00:00
}
2019-06-19 15:30:48 +00:00
MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition(
2019-03-08 19:52:21 +00:00
const SelectQueryInfo & query, const Context & context) const
2019-01-26 15:45:38 +00:00
{
2020-05-27 18:38:34 +00:00
return std::make_shared<MergeTreeIndexConditionSet>(index.name, index.sample_block, max_rows, query, context);
2019-01-26 13:12:13 +00:00
};
2019-06-19 15:30:48 +00:00
bool MergeTreeIndexSet::mayBenefitFromIndexForIn(const ASTPtr &) const
2019-02-25 08:43:19 +00:00
{
return false;
}
2020-05-28 13:45:08 +00:00
MergeTreeIndexPtr setIndexCreator(const IndexDescription & index)
2020-05-28 12:37:05 +00:00
{
size_t max_rows = index.arguments[0].get<size_t>();
return std::make_shared<MergeTreeIndexSet>(index, max_rows);
}
2020-05-28 13:09:03 +00:00
void setIndexValidator(const IndexDescription & index, bool /*attach*/)
2019-01-26 15:45:38 +00:00
{
2020-05-27 18:38:34 +00:00
if (index.arguments.size() != 1)
2019-02-13 19:26:24 +00:00
throw Exception("Set index must have exactly one argument.", ErrorCodes::INCORRECT_QUERY);
2020-05-27 18:38:34 +00:00
else if (index.arguments[0].getType() != Field::Types::UInt64)
throw Exception("Set index argument must be positive integer.", ErrorCodes::INCORRECT_QUERY);
2019-01-26 15:45:38 +00:00
}
2019-01-29 17:28:13 +00:00
}