mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
support implicit conversions between index in operator [] and key of Map
This commit is contained in:
parent
5f4ca42d15
commit
47bb5eb330
@ -1,6 +1,7 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/castTypeToEither.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
@ -95,32 +96,30 @@ private:
|
||||
|
||||
using Offsets = ColumnArray::Offsets;
|
||||
|
||||
static bool matchKeyToIndex(const IColumn & data, const Offsets & offsets,
|
||||
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs);
|
||||
static bool matchKeyToIndexNumber(
|
||||
const IColumn & data, const Offsets & offsets, bool is_key_const,
|
||||
const IColumn & index, PaddedPODArray<UInt64> & matched_idxs);
|
||||
|
||||
static bool matchKeyToIndexConst(const IColumn & data, const Offsets & offsets,
|
||||
static bool matchKeyToIndexNumberConst(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const Field & index, PaddedPODArray<UInt64> & matched_idxs);
|
||||
|
||||
template <typename DataType>
|
||||
static bool matchKeyToIndexNumber(const IColumn & data, const Offsets & offsets,
|
||||
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs);
|
||||
static bool matchKeyToIndexString(
|
||||
const IColumn & data, const Offsets & offsets, bool is_key_const,
|
||||
const IColumn & index, PaddedPODArray<UInt64> & matched_idxs);
|
||||
|
||||
template <typename DataType>
|
||||
static bool matchKeyToIndexNumberConst(const IColumn & data, const Offsets & offsets,
|
||||
const Field & index, PaddedPODArray<UInt64> & matched_idxs);
|
||||
|
||||
static bool matchKeyToIndexString(const IColumn & data, const Offsets & offsets,
|
||||
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs);
|
||||
|
||||
static bool matchKeyToIndexFixedString(const IColumn & data, const Offsets & offsets,
|
||||
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs);
|
||||
|
||||
static bool matchKeyToIndexStringConst(const IColumn & data, const Offsets & offsets,
|
||||
static bool matchKeyToIndexStringConst(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const Field & index, PaddedPODArray<UInt64> & matched_idxs);
|
||||
|
||||
template <typename Matcher>
|
||||
static void executeMatchKeyToIndex(const Offsets & offsets,
|
||||
PaddedPODArray<UInt64> & matched_idxs, const Matcher & matcher);
|
||||
|
||||
template <typename Matcher>
|
||||
static void executeMatchConstKeyToIndex(
|
||||
size_t num_rows, size_t num_values,
|
||||
PaddedPODArray<UInt64> & matched_idxs, const Matcher & matcher);
|
||||
};
|
||||
|
||||
|
||||
@ -759,23 +758,11 @@ ColumnPtr FunctionArrayElement::executeTuple(const ColumnsWithTypeAndName & argu
|
||||
namespace
|
||||
{
|
||||
|
||||
template<typename DataColumn, typename IndexColumn>
|
||||
struct MatcherString
|
||||
{
|
||||
const ColumnString & data;
|
||||
const ColumnString & index;
|
||||
|
||||
bool match(size_t row_data, size_t row_index) const
|
||||
{
|
||||
auto data_ref = data.getDataAt(row_data);
|
||||
auto index_ref = index.getDataAt(row_index);
|
||||
return memequalSmallAllowOverflow15(index_ref.data, index_ref.size, data_ref.data, data_ref.size);
|
||||
}
|
||||
};
|
||||
|
||||
struct MatcherFixedString
|
||||
{
|
||||
const ColumnFixedString & data;
|
||||
const ColumnFixedString & index;
|
||||
const DataColumn & data;
|
||||
const IndexColumn & index;
|
||||
|
||||
bool match(size_t row_data, size_t row_index) const
|
||||
{
|
||||
@ -785,9 +772,10 @@ struct MatcherFixedString
|
||||
}
|
||||
};
|
||||
|
||||
template<typename DataColumn>
|
||||
struct MatcherStringConst
|
||||
{
|
||||
const ColumnString & data;
|
||||
const DataColumn & data;
|
||||
const String & index;
|
||||
|
||||
bool match(size_t row_data, size_t /* row_index */) const
|
||||
@ -797,23 +785,23 @@ struct MatcherStringConst
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
template <typename DataType, typename IndexType>
|
||||
struct MatcherNumber
|
||||
{
|
||||
const PaddedPODArray<T> & data;
|
||||
const PaddedPODArray<T> & index;
|
||||
const PaddedPODArray<DataType> & data;
|
||||
const PaddedPODArray<IndexType> & index;
|
||||
|
||||
bool match(size_t row_data, size_t row_index) const
|
||||
{
|
||||
return data[row_data] == index[row_index];
|
||||
return data[row_data] == static_cast<DataType>(index[row_index]);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
template <typename DataType>
|
||||
struct MatcherNumberConst
|
||||
{
|
||||
const PaddedPODArray<T> & data;
|
||||
T index;
|
||||
const PaddedPODArray<DataType> & data;
|
||||
DataType index;
|
||||
|
||||
bool match(size_t row_data, size_t /* row_index */) const
|
||||
{
|
||||
@ -848,147 +836,158 @@ void FunctionArrayElement::executeMatchKeyToIndex(
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Matcher>
|
||||
void FunctionArrayElement::executeMatchConstKeyToIndex(
|
||||
size_t num_rows, size_t num_values,
|
||||
PaddedPODArray<UInt64> & matched_idxs, const Matcher & matcher)
|
||||
{
|
||||
for (size_t i = 0; i < num_rows; ++i)
|
||||
{
|
||||
bool matched = false;
|
||||
for (size_t j = 0; j < num_values; ++j)
|
||||
{
|
||||
if (matcher.match(j, i))
|
||||
{
|
||||
matched_idxs.push_back(j + 1);
|
||||
matched = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!matched)
|
||||
matched_idxs.push_back(0);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
static bool castColumnString(const IColumn * column, F && f)
|
||||
{
|
||||
return castTypeToEither<ColumnString, ColumnFixedString>(column, std::forward<F>(f));
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::matchKeyToIndexStringConst(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const Field & index, PaddedPODArray<UInt64> & matched_idxs)
|
||||
{
|
||||
const auto * data_string = checkAndGetColumn<ColumnString>(&data);
|
||||
if (!data_string)
|
||||
return false;
|
||||
return castColumnString(&data, [&](const auto & data_column)
|
||||
{
|
||||
using DataColumn = std::decay_t<decltype(data_column)>;
|
||||
|
||||
if (index.getType() != Field::Types::String)
|
||||
return false;
|
||||
|
||||
MatcherStringConst matcher{*data_string, get<const String &>(index)};
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
return true;
|
||||
MatcherStringConst<DataColumn> matcher{data_column, get<const String &>(index)};
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::matchKeyToIndexString(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs)
|
||||
const IColumn & data, const Offsets & offsets, bool is_key_const,
|
||||
const IColumn & index, PaddedPODArray<UInt64> & matched_idxs)
|
||||
{
|
||||
const auto * index_string = checkAndGetColumn<ColumnString>(arguments[1].column.get());
|
||||
if (!index_string)
|
||||
return false;
|
||||
return castColumnString(&data, [&](const auto & data_column)
|
||||
{
|
||||
return castColumnString(&index, [&](const auto & index_column)
|
||||
{
|
||||
using DataColumn = std::decay_t<decltype(data_column)>;
|
||||
using IndexColumn = std::decay_t<decltype(index_column)>;
|
||||
|
||||
const auto * data_string = checkAndGetColumn<ColumnString>(&data);
|
||||
if (!data_string)
|
||||
return false;
|
||||
MatcherString<DataColumn, IndexColumn> matcher{data_column, index_column};
|
||||
if (is_key_const)
|
||||
executeMatchConstKeyToIndex(index.size(), data.size(), matched_idxs, matcher);
|
||||
else
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
|
||||
MatcherString matcher{*data_string, *index_string};
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
return true;
|
||||
return true;
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::matchKeyToIndexFixedString(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs)
|
||||
template <typename FromType, typename ToType>
|
||||
static constexpr bool areConvertibleTypes =
|
||||
std::is_same_v<FromType, ToType>
|
||||
|| (is_integer_v<FromType> && is_integer_v<ToType>
|
||||
&& std::is_convertible_v<FromType, ToType>);
|
||||
|
||||
template <typename F>
|
||||
static bool castColumnNumeric(const IColumn * column, F && f)
|
||||
{
|
||||
const auto * index_string = checkAndGetColumn<ColumnFixedString>(arguments[1].column.get());
|
||||
if (!index_string)
|
||||
return false;
|
||||
|
||||
const auto * data_string = checkAndGetColumn<ColumnFixedString>(&data);
|
||||
if (!data_string)
|
||||
return false;
|
||||
|
||||
MatcherFixedString matcher{*data_string, *index_string};
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
return true;
|
||||
return castTypeToEither<
|
||||
ColumnVector<UInt8>,
|
||||
ColumnVector<UInt16>,
|
||||
ColumnVector<UInt32>,
|
||||
ColumnVector<UInt64>,
|
||||
ColumnVector<UInt128>,
|
||||
ColumnVector<UInt256>,
|
||||
ColumnVector<Int8>,
|
||||
ColumnVector<Int16>,
|
||||
ColumnVector<Int32>,
|
||||
ColumnVector<Int64>,
|
||||
ColumnVector<Int128>,
|
||||
ColumnVector<Int256>,
|
||||
ColumnVector<UUID>
|
||||
>(column, std::forward<F>(f));
|
||||
}
|
||||
|
||||
template <typename DataType>
|
||||
bool FunctionArrayElement::matchKeyToIndexNumberConst(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const Field & index, PaddedPODArray<UInt64> & matched_idxs)
|
||||
{
|
||||
const auto * data_numeric = checkAndGetColumn<ColumnVector<DataType>>(&data);
|
||||
if (!data_numeric)
|
||||
return false;
|
||||
|
||||
std::optional<DataType> index_as_integer;
|
||||
Field::dispatch([&](const auto & value)
|
||||
return castColumnNumeric(&data, [&](const auto & data_column)
|
||||
{
|
||||
using FieldType = std::decay_t<decltype(value)>;
|
||||
if constexpr (std::is_same_v<FieldType, DataType> || (is_integer_v<FieldType> && std::is_convertible_v<FieldType, DataType>))
|
||||
index_as_integer = static_cast<DataType>(value);
|
||||
}, index);
|
||||
using DataType = typename std::decay_t<decltype(data_column)>::ValueType;
|
||||
std::optional<DataType> index_as_integer;
|
||||
|
||||
if (!index_as_integer)
|
||||
return false;
|
||||
Field::dispatch([&](const auto & value)
|
||||
{
|
||||
using FieldType = std::decay_t<decltype(value)>;
|
||||
if constexpr (areConvertibleTypes<FieldType, DataType>)
|
||||
index_as_integer = static_cast<DataType>(value);
|
||||
}, index);
|
||||
|
||||
MatcherNumberConst<DataType> matcher{data_numeric->getData(), *index_as_integer};
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
return true;
|
||||
if (!index_as_integer)
|
||||
return false;
|
||||
|
||||
MatcherNumberConst<DataType> matcher{data_column.getData(), *index_as_integer};
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
template <typename DataType>
|
||||
bool FunctionArrayElement::matchKeyToIndexNumber(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs)
|
||||
const IColumn & data, const Offsets & offsets, bool is_key_const,
|
||||
const IColumn & index, PaddedPODArray<UInt64> & matched_idxs)
|
||||
{
|
||||
const auto * index_numeric = checkAndGetColumn<ColumnVector<DataType>>(arguments[1].column.get());
|
||||
if (!index_numeric)
|
||||
return false;
|
||||
return castColumnNumeric(&data, [&](const auto & data_column)
|
||||
{
|
||||
return castColumnNumeric(&index, [&](const auto & index_column)
|
||||
{
|
||||
using DataType = typename std::decay_t<decltype(data_column)>::ValueType;
|
||||
using IndexType = typename std::decay_t<decltype(index_column)>::ValueType;
|
||||
|
||||
const auto * data_numeric = checkAndGetColumn<ColumnVector<DataType>>(&data);
|
||||
if (!data_numeric)
|
||||
return false;
|
||||
if constexpr (areConvertibleTypes<IndexType, DataType>)
|
||||
{
|
||||
MatcherNumber<DataType, IndexType> matcher{data_column.getData(), index_column.getData()};
|
||||
if (is_key_const)
|
||||
executeMatchConstKeyToIndex(index_column.size(), data_column.size(), matched_idxs, matcher);
|
||||
else
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
|
||||
MatcherNumber<DataType> matcher{data_numeric->getData(), index_numeric->getData()};
|
||||
executeMatchKeyToIndex(offsets, matched_idxs, matcher);
|
||||
return true;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::matchKeyToIndex(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const ColumnsWithTypeAndName & arguments, PaddedPODArray<UInt64> & matched_idxs)
|
||||
{
|
||||
return matchKeyToIndexNumber<UInt8>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UInt16>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UInt32>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UInt64>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UInt128>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UInt256>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int8>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int16>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int32>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int64>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int128>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int256>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UInt256>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UUID>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexString(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexFixedString(data, offsets, arguments, matched_idxs);
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::matchKeyToIndexConst(
|
||||
const IColumn & data, const Offsets & offsets,
|
||||
const Field & index, PaddedPODArray<UInt64> & matched_idxs)
|
||||
{
|
||||
return matchKeyToIndexNumberConst<UInt8>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<UInt16>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<UInt32>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<UInt64>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<UInt128>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<UInt256>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int8>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int16>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int32>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int64>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int128>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int256>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<UUID>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexStringConst(data, offsets, index, matched_idxs);
|
||||
return false;
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
ColumnPtr FunctionArrayElement::executeMap(
|
||||
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
|
||||
{
|
||||
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(arguments[0].column.get());
|
||||
if (!col_map)
|
||||
return nullptr;
|
||||
const auto * col_map = checkAndGetColumn<ColumnMap>(arguments[0].column.get());
|
||||
const auto * col_const_map = checkAndGetColumnConst<ColumnMap>(arguments[0].column.get());
|
||||
assert(col_map || col_const_map);
|
||||
|
||||
if (col_const_map)
|
||||
col_map = typeid_cast<const ColumnMap *>(&col_const_map->getDataColumn());
|
||||
|
||||
const auto & nested_column = col_map->getNestedColumn();
|
||||
const auto & keys_data = col_map->getNestedData().getColumn(0);
|
||||
@ -1000,29 +999,33 @@ ColumnPtr FunctionArrayElement::executeMap(
|
||||
indices_column->reserve(input_rows_count);
|
||||
auto & indices_data = assert_cast<ColumnVector<UInt64> &>(*indices_column).getData();
|
||||
|
||||
bool executed = false;
|
||||
if (!isColumnConst(*arguments[1].column))
|
||||
{
|
||||
if (input_rows_count > 0 && !matchKeyToIndex(keys_data, offsets, arguments, indices_data))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal types of arguments: {}, {} for function {}",
|
||||
arguments[0].type->getName(), arguments[1].type->getName(), getName());
|
||||
executed = matchKeyToIndexNumber(keys_data, offsets, !!col_const_map, *arguments[1].column, indices_data)
|
||||
|| matchKeyToIndexString(keys_data, offsets, !!col_const_map, *arguments[1].column, indices_data);
|
||||
}
|
||||
else
|
||||
{
|
||||
Field index = (*arguments[1].column)[0];
|
||||
|
||||
// Get Matched key's value
|
||||
if (input_rows_count > 0 && !matchKeyToIndexConst(keys_data, offsets, index, indices_data))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal types of arguments: {}, {} for function {}",
|
||||
arguments[0].type->getName(), arguments[1].type->getName(), getName());
|
||||
executed = matchKeyToIndexNumberConst(keys_data, offsets, index, indices_data)
|
||||
|| matchKeyToIndexStringConst(keys_data, offsets, index, indices_data);
|
||||
}
|
||||
|
||||
if (!executed)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal types of arguments: {}, {} for function {}",
|
||||
arguments[0].type->getName(), arguments[1].type->getName(), getName());
|
||||
|
||||
ColumnPtr values_array = ColumnArray::create(values_data.getPtr(), nested_column.getOffsetsPtr());
|
||||
if (col_const_map)
|
||||
values_array = ColumnConst::create(values_array, input_rows_count);
|
||||
|
||||
/// Prepare arguments to call arrayElement for array with values and calculated indices at previous step.
|
||||
ColumnsWithTypeAndName new_arguments =
|
||||
{
|
||||
{
|
||||
ColumnArray::create(values_data.getPtr(), nested_column.getOffsetsPtr()),
|
||||
values_array,
|
||||
std::make_shared<DataTypeArray>(result_type),
|
||||
""
|
||||
},
|
||||
@ -1066,13 +1069,14 @@ DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments)
|
||||
|
||||
ColumnPtr FunctionArrayElement::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
|
||||
{
|
||||
/// Check nullability.
|
||||
bool is_array_of_nullable = false;
|
||||
const auto * col_map = checkAndGetColumn<ColumnMap>(arguments[0].column.get());
|
||||
const auto * col_const_map = checkAndGetColumnConst<ColumnMap>(arguments[0].column.get());
|
||||
|
||||
const ColumnMap * col_map = checkAndGetColumn<ColumnMap>(arguments[0].column.get());
|
||||
if (col_map)
|
||||
if (col_map || col_const_map)
|
||||
return executeMap(arguments, result_type, input_rows_count);
|
||||
|
||||
/// Check nullability.
|
||||
bool is_array_of_nullable = false;
|
||||
const ColumnArray * col_array = nullptr;
|
||||
const ColumnArray * col_const_array = nullptr;
|
||||
|
||||
|
22
tests/queries/0_stateless/02014_map_different_keys.reference
Normal file
22
tests/queries/0_stateless/02014_map_different_keys.reference
Normal file
@ -0,0 +1,22 @@
|
||||
...const maps...
|
||||
0
|
||||
2
|
||||
0
|
||||
4
|
||||
0
|
||||
0
|
||||
2
|
||||
0
|
||||
4
|
||||
0
|
||||
4
|
||||
4
|
||||
...int keys...
|
||||
foo bar bar
|
||||
foo foo
|
||||
...string keys...
|
||||
foo foo
|
||||
foo foo
|
||||
foo foo
|
||||
bar bar
|
||||
0
|
32
tests/queries/0_stateless/02014_map_different_keys.sql
Normal file
32
tests/queries/0_stateless/02014_map_different_keys.sql
Normal file
@ -0,0 +1,32 @@
|
||||
SELECT '...const maps...';
|
||||
|
||||
WITH map(1, 2, 3, 4) AS m SELECT m[number] FROM numbers(5);
|
||||
WITH map('1', 2, '3', 4) AS m SELECT m[toString(number)] FROM numbers(5);
|
||||
|
||||
WITH map(1, 2, 3, 4) AS m SELECT m[3];
|
||||
WITH map('1', 2, '3', 4) AS m SELECT m['3'];
|
||||
|
||||
DROP TABLE IF EXISTS t_map_02014;
|
||||
|
||||
CREATE TABLE t_map_02014(i1 UInt64, i2 Int32, m1 Map(UInt32, String), m2 Map(Int8, String), m3 Map(Int128, String)) ENGINE = Memory;
|
||||
INSERT INTO t_map_02014 VALUES (1, -1, map(1, 'foo', 2, 'bar'), map(-1, 'foo', 1, 'bar'), map(-1, 'foo', 1, 'bar'));
|
||||
|
||||
SELECT '...int keys...';
|
||||
|
||||
SELECT m1[i1], m2[i1], m3[i1] FROM t_map_02014;
|
||||
SELECT m1[i2], m2[i2], m3[i2] FROM t_map_02014;
|
||||
|
||||
DROP TABLE IF EXISTS t_map_02014;
|
||||
|
||||
CREATE TABLE t_map_02014(s String, fs FixedString(3), m1 Map(String, String), m2 Map(FixedString(3), String)) ENGINE = Memory;
|
||||
INSERT INTO t_map_02014 VALUES ('aaa', 'bbb', map('aaa', 'foo', 'bbb', 'bar'), map('aaa', 'foo', 'bbb', 'bar'));
|
||||
|
||||
SELECT '...string keys...';
|
||||
|
||||
SELECT m1['aaa'], m2['aaa'] FROM t_map_02014;
|
||||
SELECT m1['aaa'::FixedString(3)], m2['aaa'::FixedString(3)] FROM t_map_02014;
|
||||
SELECT m1[s], m2[s] FROM t_map_02014;
|
||||
SELECT m1[fs], m2[fs] FROM t_map_02014;
|
||||
SELECT length(m2['aaa'::FixedString(4)]) FROM t_map_02014;
|
||||
|
||||
DROP TABLE IF EXISTS t_map_02014;
|
Loading…
Reference in New Issue
Block a user