mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 02:21:59 +00:00
optimize function mapUpdate
This commit is contained in:
parent
0f01725d8b
commit
6d05968a0f
@ -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);
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user