optimize function mapUpdate

This commit is contained in:
Anton Popov 2023-03-28 16:23:32 +00:00
parent 0f01725d8b
commit 6d05968a0f

View File

@ -18,6 +18,7 @@
#include "array/arrayIndex.h"
#include "Functions/like.h"
#include "Functions/FunctionsStringSearch.h"
#include <Common/HashTable/HashSet.h>
namespace DB
@ -616,103 +617,110 @@ public:
"Number of arguments for function {} doesn't match: passed {}, should be 2",
getName(), arguments.size());
const DataTypeMap * left = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
const DataTypeMap * right = checkAndGetDataType<DataTypeMap>(arguments[1].type.get());
const auto * left = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
const auto * right = checkAndGetDataType<DataTypeMap>(arguments[1].type.get());
if (!left || !right)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The two arguments for function {} must be both Map type",
getName());
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The two arguments for function {} must be both Map type", getName());
if (!left->getKeyType()->equals(*right->getKeyType()) || !left->getValueType()->equals(*right->getValueType()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The Key And Value type of Map for function {} must be the same",
getName());
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The Key And Value type of Map for function {} must be the same", getName());
return std::make_shared<DataTypeMap>(left->getKeyType(), left->getValueType());
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const ColumnMap * col_map_left = typeid_cast<const ColumnMap *>(arguments[0].column.get());
const auto * col_const_map_left = checkAndGetColumnConst<ColumnMap>(arguments[0].column.get());
bool col_const_map_left_flag = false;
if (col_const_map_left)
{
col_const_map_left_flag = true;
col_map_left = typeid_cast<const ColumnMap *>(&col_const_map_left->getDataColumn());
}
if (!col_map_left)
return nullptr;
bool is_left_const = isColumnConst(*arguments[0].column);
bool is_right_const = isColumnConst(*arguments[1].column);
const ColumnMap * col_map_right = typeid_cast<const ColumnMap *>(arguments[1].column.get());
const auto * col_const_map_right = checkAndGetColumnConst<ColumnMap>(arguments[1].column.get());
bool col_const_map_right_flag = false;
if (col_const_map_right)
{
col_const_map_right_flag = true;
col_map_right = typeid_cast<const ColumnMap *>(&col_const_map_right->getDataColumn());
}
if (!col_map_right)
return nullptr;
const auto * map_column_left = is_left_const
? checkAndGetColumnConstData<ColumnMap>(arguments[0].column.get())
: checkAndGetColumn<ColumnMap>(arguments[0].column.get());
const auto & nested_column_left = col_map_left->getNestedColumn();
const auto & keys_data_left = col_map_left->getNestedData().getColumn(0);
const auto & values_data_left = col_map_left->getNestedData().getColumn(1);
const auto * map_column_right = is_right_const
? checkAndGetColumnConstData<ColumnMap>(arguments[1].column.get())
: checkAndGetColumn<ColumnMap>(arguments[1].column.get());
if (!map_column_left || !map_column_right)
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Arguments for function {} must be maps, got {} and {} instead",
getName(), arguments[0].column->getName(), arguments[1].column->getName());
const auto & nested_column_left = map_column_left->getNestedColumn();
const auto & keys_data_left = map_column_left->getNestedData().getColumn(0);
const auto & values_data_left = map_column_left->getNestedData().getColumn(1);
const auto & offsets_left = nested_column_left.getOffsets();
const auto & nested_column_right = col_map_right->getNestedColumn();
const auto & keys_data_right = col_map_right->getNestedData().getColumn(0);
const auto & values_data_right = col_map_right->getNestedData().getColumn(1);
const auto & nested_column_right = map_column_right->getNestedColumn();
const auto & keys_data_right = map_column_right->getNestedData().getColumn(0);
const auto & values_data_right = map_column_right->getNestedData().getColumn(1);
const auto & offsets_right = nested_column_right.getOffsets();
const auto & result_type_map = static_cast<const DataTypeMap &>(*result_type);
const DataTypePtr & key_type = result_type_map.getKeyType();
const DataTypePtr & value_type = result_type_map.getValueType();
MutableColumnPtr keys_data = key_type->createColumn();
MutableColumnPtr values_data = value_type->createColumn();
MutableColumnPtr offsets = DataTypeNumber<IColumn::Offset>().createColumn();
auto result_keys = keys_data_left.cloneEmpty();
auto result_values = values_data_left.cloneEmpty();
auto result_offsets = ColumnVector<IColumn::Offset>::create(input_rows_count);
auto & result_offsets_data = result_offsets->getData();
using Set = HashSetWithStackMemory<StringRef, StringRefHash, 4>;
Set right_keys_const;
if (is_right_const)
{
for (size_t i = 0; i < keys_data_right.size(); ++i)
right_keys_const.insert(keys_data_right.getDataAt(i));
}
IColumn::Offset current_offset = 0;
for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx)
{
size_t left_it_begin = col_const_map_left_flag ? 0 : offsets_left[row_idx - 1];
size_t left_it_end = col_const_map_left_flag ? offsets_left.size() : offsets_left[row_idx];
size_t right_it_begin = col_const_map_right_flag ? 0 : offsets_right[row_idx - 1];
size_t right_it_end = col_const_map_right_flag ? offsets_right.size() : offsets_right[row_idx];
size_t left_from = is_left_const ? 0 : offsets_left[row_idx - 1];
size_t left_to = is_left_const ? offsets_left[0] : offsets_left[row_idx];
for (size_t i = left_it_begin; i < left_it_end; ++i)
size_t right_from = is_right_const ? 0 : offsets_right[row_idx - 1];
size_t right_to = is_right_const ? offsets_right[0] : offsets_right[row_idx];
auto execute_row = [&](const auto & set)
{
bool matched = false;
auto key = keys_data_left.getDataAt(i);
for (size_t j = right_it_begin; j < right_it_end; ++j)
for (size_t i = left_from; i < left_to; ++i)
{
if (keys_data_right.getDataAt(j).toString() == key.toString())
if (!set.find(keys_data_left.getDataAt(i)))
{
matched = true;
break;
result_keys->insertFrom(keys_data_left, i);
result_values->insertFrom(values_data_left, i);
++current_offset;
}
}
if (!matched)
{
keys_data->insertFrom(keys_data_left, i);
values_data->insertFrom(values_data_left, i);
++current_offset;
}
}
};
for (size_t j = right_it_begin; j < right_it_end; ++j)
if (is_right_const)
{
keys_data->insertFrom(keys_data_right, j);
values_data->insertFrom(values_data_right, j);
++current_offset;
execute_row(right_keys_const);
}
else
{
Set right_keys;
for (size_t i = right_from; i < right_to; ++i)
right_keys.insert(keys_data_right.getDataAt(i));
execute_row(right_keys);
}
offsets->insert(current_offset);
size_t right_map_size = right_to - right_from;
result_keys->insertRangeFrom(keys_data_right, right_from, right_map_size);
result_values->insertRangeFrom(values_data_right, right_from, right_map_size);
current_offset += right_map_size;
result_offsets_data[row_idx] = current_offset;
}
auto nested_column = ColumnArray::create(
ColumnTuple::create(Columns{std::move(keys_data), std::move(values_data)}),
std::move(offsets));
ColumnTuple::create(Columns{std::move(result_keys), std::move(result_values)}),
std::move(result_offsets));
return ColumnMap::create(nested_column);
}