2020-02-05 10:12:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
#include <DataTypes/DataTypeArray.h>
|
2022-11-15 01:42:46 +00:00
|
|
|
#include <DataTypes/DataTypeMap.h>
|
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnMap.h>
|
2020-02-05 10:12:19 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/ArrayJoinAction.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2023-02-27 08:31:40 +00:00
|
|
|
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
|
2020-02-05 10:12:19 +00:00
|
|
|
extern const int TYPE_MISMATCH;
|
|
|
|
}
|
|
|
|
|
2022-12-14 13:02:03 +00:00
|
|
|
std::shared_ptr<const DataTypeArray> getArrayJoinDataType(DataTypePtr type)
|
2022-11-15 01:42:46 +00:00
|
|
|
{
|
2022-11-15 07:26:36 +00:00
|
|
|
if (const auto * array_type = typeid_cast<const DataTypeArray *>(type.get()))
|
2022-12-14 13:02:03 +00:00
|
|
|
return std::shared_ptr<const DataTypeArray>{type, array_type};
|
2022-11-21 10:03:46 +00:00
|
|
|
else if (const auto * map_type = typeid_cast<const DataTypeMap *>(type.get()))
|
2022-11-15 01:42:46 +00:00
|
|
|
{
|
2022-11-21 10:03:46 +00:00
|
|
|
const auto & nested_type = map_type->getNestedType();
|
2022-12-14 13:02:03 +00:00
|
|
|
const auto * nested_array_type = typeid_cast<const DataTypeArray *>(nested_type.get());
|
|
|
|
return std::shared_ptr<const DataTypeArray>{nested_type, nested_array_type};
|
2022-11-15 01:42:46 +00:00
|
|
|
}
|
2022-11-21 10:03:46 +00:00
|
|
|
else
|
|
|
|
return nullptr;
|
2022-11-15 01:42:46 +00:00
|
|
|
}
|
|
|
|
|
2022-12-14 13:02:03 +00:00
|
|
|
ColumnPtr getArrayJoinColumn(const ColumnPtr & column)
|
2022-11-15 01:42:46 +00:00
|
|
|
{
|
2022-12-14 13:02:03 +00:00
|
|
|
if (typeid_cast<const ColumnArray *>(column.get()))
|
|
|
|
return column;
|
2022-11-21 10:03:46 +00:00
|
|
|
else if (const auto * map = typeid_cast<const ColumnMap *>(column.get()))
|
2022-12-14 13:02:03 +00:00
|
|
|
return map->getNestedColumnPtr();
|
2022-11-21 10:03:46 +00:00
|
|
|
else
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
|
2022-12-14 13:02:03 +00:00
|
|
|
const ColumnArray * getArrayJoinColumnRawPtr(const ColumnPtr & column)
|
|
|
|
{
|
|
|
|
if (const auto & col_arr = getArrayJoinColumn(column))
|
|
|
|
return typeid_cast<const ColumnArray *>(col_arr.get());
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
|
2022-11-21 10:03:46 +00:00
|
|
|
ColumnWithTypeAndName convertArrayJoinColumn(const ColumnWithTypeAndName & src_col)
|
|
|
|
{
|
|
|
|
ColumnWithTypeAndName array_col;
|
|
|
|
array_col.name = src_col.name;
|
2022-12-14 13:02:03 +00:00
|
|
|
array_col.type = getArrayJoinDataType(src_col.type);
|
|
|
|
array_col.column = getArrayJoinColumn(src_col.column->convertToFullColumnIfConst());
|
2022-11-21 10:03:46 +00:00
|
|
|
return array_col;
|
2022-11-15 01:42:46 +00:00
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
ArrayJoinAction::ArrayJoinAction(const NameSet & array_joined_columns_, bool array_join_is_left, ContextPtr context)
|
2020-02-05 10:12:19 +00:00
|
|
|
: columns(array_joined_columns_)
|
|
|
|
, is_left(array_join_is_left)
|
2021-04-10 23:33:54 +00:00
|
|
|
, is_unaligned(context->getSettingsRef().enable_unaligned_array_join)
|
2023-09-15 08:33:48 +00:00
|
|
|
, max_block_size(context->getSettingsRef().max_block_size)
|
2020-02-05 10:12:19 +00:00
|
|
|
{
|
|
|
|
if (columns.empty())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "No arrays to join");
|
2020-02-05 10:12:19 +00:00
|
|
|
|
|
|
|
if (is_unaligned)
|
|
|
|
{
|
|
|
|
function_length = FunctionFactory::instance().get("length", context);
|
|
|
|
function_greatest = FunctionFactory::instance().get("greatest", context);
|
2022-11-15 01:42:46 +00:00
|
|
|
function_array_resize = FunctionFactory::instance().get("arrayResize", context);
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
|
|
|
else if (is_left)
|
|
|
|
function_builder = FunctionFactory::instance().get("emptyArrayToSingle", context);
|
|
|
|
}
|
|
|
|
|
2020-09-10 18:36:51 +00:00
|
|
|
void ArrayJoinAction::prepare(ColumnsWithTypeAndName & sample) const
|
2020-02-05 10:12:19 +00:00
|
|
|
{
|
2020-09-10 18:36:51 +00:00
|
|
|
for (auto & current : sample)
|
2020-02-05 10:12:19 +00:00
|
|
|
{
|
2022-04-18 10:18:43 +00:00
|
|
|
if (!columns.contains(current.name))
|
2020-09-10 18:36:51 +00:00
|
|
|
continue;
|
|
|
|
|
2022-12-14 13:02:03 +00:00
|
|
|
if (const auto & type = getArrayJoinDataType(current.type))
|
2022-11-15 01:42:46 +00:00
|
|
|
{
|
|
|
|
current.column = nullptr;
|
|
|
|
current.type = type->getNestedType();
|
|
|
|
}
|
|
|
|
else
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "ARRAY JOIN requires array or map argument");
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-09-15 08:33:48 +00:00
|
|
|
ArrayJoinResultIteratorPtr ArrayJoinAction::execute(Block block)
|
2020-02-05 10:12:19 +00:00
|
|
|
{
|
|
|
|
if (columns.empty())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "No arrays to join");
|
2020-02-05 10:12:19 +00:00
|
|
|
|
2023-09-15 08:33:48 +00:00
|
|
|
return std::make_unique<ArrayJoinResultIterator>(this, std::move(block));
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
ArrayJoinResultIterator::ArrayJoinResultIterator(const ArrayJoinAction * array_join_, Block block_)
|
|
|
|
: array_join(array_join_), block(std::move(block_)), total_rows(block.rows()), current_row(0)
|
|
|
|
{
|
|
|
|
const auto & columns = array_join->columns;
|
|
|
|
bool is_unaligned = array_join->is_unaligned;
|
|
|
|
bool is_left = array_join->is_left;
|
|
|
|
const auto & function_length = array_join->function_length;
|
|
|
|
const auto & function_greatest = array_join->function_greatest;
|
|
|
|
const auto & function_array_resize = array_join->function_array_resize;
|
|
|
|
const auto & function_builder = array_join->function_builder;
|
|
|
|
|
|
|
|
any_array_map_ptr = block.getByName(*columns.begin()).column->convertToFullColumnIfConst();
|
|
|
|
any_array = getArrayJoinColumnRawPtr(any_array_map_ptr);
|
2020-02-05 10:12:19 +00:00
|
|
|
if (!any_array)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "ARRAY JOIN requires array or map argument");
|
2020-02-05 10:12:19 +00:00
|
|
|
|
|
|
|
if (is_unaligned)
|
|
|
|
{
|
|
|
|
/// Resize all array joined columns to the longest one, (at least 1 if LEFT ARRAY JOIN), padded with default values.
|
|
|
|
auto rows = block.rows();
|
|
|
|
auto uint64 = std::make_shared<DataTypeUInt64>();
|
2020-10-19 18:37:44 +00:00
|
|
|
ColumnWithTypeAndName column_of_max_length{{}, uint64, {}};
|
2020-02-05 10:12:19 +00:00
|
|
|
if (is_left)
|
|
|
|
column_of_max_length = ColumnWithTypeAndName(uint64->createColumnConst(rows, 1u), uint64, {});
|
|
|
|
else
|
|
|
|
column_of_max_length = ColumnWithTypeAndName(uint64->createColumnConst(rows, 0u), uint64, {});
|
|
|
|
|
|
|
|
for (const auto & name : columns)
|
|
|
|
{
|
|
|
|
auto & src_col = block.getByName(name);
|
|
|
|
|
2022-11-21 10:03:46 +00:00
|
|
|
ColumnWithTypeAndName array_col = convertArrayJoinColumn(src_col);
|
2022-11-15 07:26:36 +00:00
|
|
|
ColumnsWithTypeAndName tmp_block{array_col}; //, {{}, uint64, {}}};
|
2020-10-19 18:37:44 +00:00
|
|
|
auto len_col = function_length->build(tmp_block)->execute(tmp_block, uint64, rows);
|
2020-02-05 10:12:19 +00:00
|
|
|
|
2020-10-19 18:37:44 +00:00
|
|
|
ColumnsWithTypeAndName tmp_block2{column_of_max_length, {len_col, uint64, {}}};
|
|
|
|
column_of_max_length.column = function_greatest->build(tmp_block2)->execute(tmp_block2, uint64, rows);
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto & name : columns)
|
|
|
|
{
|
|
|
|
auto & src_col = block.getByName(name);
|
|
|
|
|
2022-11-21 10:03:46 +00:00
|
|
|
ColumnWithTypeAndName array_col = convertArrayJoinColumn(src_col);
|
2022-11-15 07:26:36 +00:00
|
|
|
ColumnsWithTypeAndName tmp_block{array_col, column_of_max_length};
|
|
|
|
array_col.column = function_array_resize->build(tmp_block)->execute(tmp_block, array_col.type, rows);
|
|
|
|
|
|
|
|
src_col = std::move(array_col);
|
2022-11-15 01:42:46 +00:00
|
|
|
any_array_map_ptr = src_col.column->convertToFullColumnIfConst();
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
|
|
|
|
2022-12-14 13:02:03 +00:00
|
|
|
any_array = getArrayJoinColumnRawPtr(any_array_map_ptr);
|
2022-11-15 01:42:46 +00:00
|
|
|
if (!any_array)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "ARRAY JOIN requires array or map argument");
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
|
|
|
else if (is_left)
|
|
|
|
{
|
|
|
|
for (const auto & name : columns)
|
|
|
|
{
|
2022-11-15 01:42:46 +00:00
|
|
|
const auto & src_col = block.getByName(name);
|
2022-11-21 10:03:46 +00:00
|
|
|
ColumnWithTypeAndName array_col = convertArrayJoinColumn(src_col);
|
2022-11-15 01:42:46 +00:00
|
|
|
ColumnsWithTypeAndName tmp_block{array_col};
|
|
|
|
non_empty_array_columns[name] = function_builder->build(tmp_block)->execute(tmp_block, array_col.type, array_col.column->size());
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
|
|
|
|
2022-11-15 01:42:46 +00:00
|
|
|
any_array_map_ptr = non_empty_array_columns.begin()->second->convertToFullColumnIfConst();
|
2022-12-14 13:02:03 +00:00
|
|
|
any_array = getArrayJoinColumnRawPtr(any_array_map_ptr);
|
2022-11-15 01:42:46 +00:00
|
|
|
if (!any_array)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "ARRAY JOIN requires array or map argument");
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
2023-09-15 08:33:48 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool ArrayJoinResultIterator::hasNext() const
|
|
|
|
{
|
|
|
|
return total_rows != 0 && current_row < total_rows;
|
|
|
|
}
|
2020-02-05 10:12:19 +00:00
|
|
|
|
2022-11-15 01:42:46 +00:00
|
|
|
|
2023-09-15 08:33:48 +00:00
|
|
|
Block ArrayJoinResultIterator::next()
|
|
|
|
{
|
|
|
|
if (!hasNext())
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "No more elements in ArrayJoinResultIterator.");
|
|
|
|
|
|
|
|
size_t max_block_size = array_join->max_block_size;
|
|
|
|
const auto & offsets = any_array->getOffsets();
|
|
|
|
|
|
|
|
/// Make sure output block rows do not exceed max_block_size.
|
|
|
|
size_t next_row = current_row;
|
|
|
|
for (; next_row < total_rows; ++next_row)
|
|
|
|
{
|
|
|
|
if (offsets[next_row] - offsets[current_row - 1] >= max_block_size)
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
if (next_row == current_row)
|
|
|
|
++next_row;
|
|
|
|
|
|
|
|
Block res;
|
2020-02-05 10:12:19 +00:00
|
|
|
size_t num_columns = block.columns();
|
2023-09-15 08:33:48 +00:00
|
|
|
const auto & columns = array_join->columns;
|
|
|
|
bool is_unaligned = array_join->is_unaligned;
|
|
|
|
bool is_left = array_join->is_left;
|
|
|
|
auto cut_any_col = any_array->cut(current_row, next_row - current_row);
|
|
|
|
const auto * cut_any_array = typeid_cast<const ColumnArray *>(cut_any_col.get());
|
2023-11-24 07:17:39 +00:00
|
|
|
|
2020-02-05 10:12:19 +00:00
|
|
|
for (size_t i = 0; i < num_columns; ++i)
|
|
|
|
{
|
2023-09-15 08:33:48 +00:00
|
|
|
ColumnWithTypeAndName current = block.safeGetByPosition(i);
|
2023-11-24 07:17:39 +00:00
|
|
|
|
|
|
|
/// Reuse cut_any_col if possible to avoid unnecessary cut.
|
|
|
|
if (!is_unaligned && !is_left && current.name == *columns.begin())
|
2023-11-27 02:48:01 +00:00
|
|
|
{
|
2023-11-24 07:17:39 +00:00
|
|
|
current.column = cut_any_col;
|
2023-11-27 02:48:01 +00:00
|
|
|
current.type = getArrayJoinDataType(current.type);
|
|
|
|
}
|
2023-11-24 07:17:39 +00:00
|
|
|
else
|
|
|
|
current.column = current.column->cut(current_row, next_row - current_row);
|
2020-02-05 10:12:19 +00:00
|
|
|
|
2022-04-18 10:18:43 +00:00
|
|
|
if (columns.contains(current.name))
|
2020-02-05 10:12:19 +00:00
|
|
|
{
|
2022-12-14 13:02:03 +00:00
|
|
|
if (const auto & type = getArrayJoinDataType(current.type))
|
2022-11-15 01:42:46 +00:00
|
|
|
{
|
2022-11-15 07:26:36 +00:00
|
|
|
ColumnPtr array_ptr;
|
|
|
|
if (typeid_cast<const DataTypeArray *>(current.type.get()))
|
2022-11-15 01:42:46 +00:00
|
|
|
{
|
2023-09-15 08:33:48 +00:00
|
|
|
array_ptr = (is_left && !is_unaligned) ? non_empty_array_columns[current.name]->cut(current_row, next_row - current_row)
|
|
|
|
: current.column;
|
2022-11-15 01:42:46 +00:00
|
|
|
array_ptr = array_ptr->convertToFullColumnIfConst();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
ColumnPtr map_ptr = current.column->convertToFullColumnIfConst();
|
|
|
|
const ColumnMap & map = typeid_cast<const ColumnMap &>(*map_ptr);
|
2023-09-15 08:33:48 +00:00
|
|
|
array_ptr = (is_left && !is_unaligned) ? non_empty_array_columns[current.name]->cut(current_row, next_row - current_row)
|
|
|
|
: map.getNestedColumnPtr();
|
2022-11-15 01:42:46 +00:00
|
|
|
}
|
2022-11-15 07:26:36 +00:00
|
|
|
|
|
|
|
const ColumnArray & array = typeid_cast<const ColumnArray &>(*array_ptr);
|
2023-09-15 08:33:48 +00:00
|
|
|
if (!is_unaligned && !array.hasEqualOffsets(*cut_any_array))
|
2023-02-27 08:31:40 +00:00
|
|
|
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Sizes of ARRAY-JOIN-ed arrays do not match");
|
2022-11-15 07:26:36 +00:00
|
|
|
|
|
|
|
current.column = typeid_cast<const ColumnArray &>(*array_ptr).getDataPtr();
|
|
|
|
current.type = type->getNestedType();
|
2022-11-15 01:42:46 +00:00
|
|
|
}
|
|
|
|
else
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TYPE_MISMATCH, "ARRAY JOIN of not array nor map: {}", current.name);
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2023-09-15 08:33:48 +00:00
|
|
|
current.column = current.column->replicate(cut_any_array->getOffsets());
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
2023-09-15 08:33:48 +00:00
|
|
|
|
|
|
|
res.insert(std::move(current));
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
2023-09-15 08:33:48 +00:00
|
|
|
|
|
|
|
current_row = next_row;
|
|
|
|
return res;
|
2020-02-05 10:12:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|