updated interface, bug investigation

This commit is contained in:
myrrc 2020-07-16 23:59:32 +03:00
parent bc8e8c25e9
commit 0ce74ccc28
5 changed files with 73 additions and 64 deletions

View File

@ -15,6 +15,13 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
/**
* How data is stored (in a nutshell):
* we have a dictionary @e reverse_index in ColumnUnique that holds pairs (DataType, UIntXX) and a column
* with UIntXX holding actual data indices.
* To obtain the value's index, call #getOrFindIndex.
* To operate on the data (so called indices column), call #getIndexes.
*/
class ColumnLowCardinality final : public COWHelper<IColumn, ColumnLowCardinality> class ColumnLowCardinality final : public COWHelper<IColumn, ColumnLowCardinality>
{ {
friend class COWHelper<IColumn, ColumnLowCardinality>; friend class COWHelper<IColumn, ColumnLowCardinality>;

View File

@ -112,9 +112,22 @@ public:
UInt128 getHash() const override { return hash.getHash(*getRawColumnPtr()); } UInt128 getHash() const override { return hash.getHash(*getRawColumnPtr()); }
inline UInt64 getIndexByValue(const StringRef& value) const override inline UInt64 getOrFindIndex(const StringRef& value) const override
{ {
return reverse_index.getInsertionPointConst(value); if (std::optional<UInt64> res = reverse_index.getIndex(value); res)
return res.value();
auto& nested = *getNestedColumn();
for (size_t i = 0; i < nested.size(); ++i) {
std::cout << nested.getDataAt(i) << std::endl;
if (nested.getDataAt(i) == value)
return i;
};
throw Exception(
"Trying to find the value that is not present in the index",
ErrorCodes::LOGICAL_ERROR);
} }
private: private:

View File

@ -71,7 +71,11 @@ public:
/** /**
* Given some value (usually, of type @e ColumnType) @p value that is convertible to DB::StringRef, obtains its * Given some value (usually, of type @e ColumnType) @p value that is convertible to DB::StringRef, obtains its
* index in the DB::ColumnUnique::reverse_index hastable. * index in the DB::ColumnUnique::reverse_index hashtable (std::nullopt if not found).
*
* The reverse index (StringRef => UInt64) is built lazily, so there are two variants:
* - On the function call it's present. Therefore we obtain the index in O(1).
* - The reverse index is absent. We find the index in O(dictionary size) by performing the linear search.
* *
* @see DB::ReverseIndex * @see DB::ReverseIndex
* @see DB::ColumnUnique * @see DB::ColumnUnique
@ -81,7 +85,7 @@ public:
* region, so it can be easily represented as a @e StringRef. So we pass that ref to this function and get its * region, so it can be easily represented as a @e StringRef. So we pass that ref to this function and get its
* index in the dictionary, which can be used to operate with the indices column. * index in the dictionary, which can be used to operate with the indices column.
*/ */
virtual inline UInt64 getIndexByValue(const StringRef& value) const = 0; virtual inline UInt64 getOrFindIndex(const StringRef& value) const = 0;
void insert(const Field &) override void insert(const Field &) override
{ {

View File

@ -329,8 +329,8 @@ public:
/// If index is not built, builds it. /// If index is not built, builds it.
UInt64 getInsertionPoint(const StringRef & data); UInt64 getInsertionPoint(const StringRef & data);
/// If index is not found, throws a ErrorCodes::LOGICAL_ERROR /// Returns the found index if the #index is built, otherwise, searches for it linearly.
UInt64 getInsertionPointConst(const StringRef & data) const; std::optional<UInt64> getIndex(const StringRef & data) const;
UInt64 lastInsertionPoint() const { return size() + base_index; } UInt64 lastInsertionPoint() const { return size() + base_index; }
@ -521,10 +521,10 @@ UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPoint(const StringRef &
} }
template <typename IndexType, typename ColumnType> template <typename IndexType, typename ColumnType>
UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPointConst(const StringRef & data) const std::optional<UInt64> ReverseIndex<IndexType, ColumnType>::getIndex(const StringRef & data) const
{ {
if (!index) if (!index)
throw Exception("No built index in ReverseIndex", ErrorCodes::LOGICAL_ERROR); return {};
using IteratorType = typename IndexMapType::iterator; using IteratorType = typename IndexMapType::iterator;
IteratorType iterator; IteratorType iterator;

View File

@ -634,7 +634,7 @@ inline bool allowNonLowCardinalityArg(const DataTypePtr& arr, const DataTypePtr&
} }
template <typename IndexConv, typename Name> template <class ConcreteAction, class Name>
class FunctionArrayIndex : public IFunction class FunctionArrayIndex : public IFunction
{ {
public: public:
@ -642,7 +642,7 @@ public:
static FunctionPtr create(const Context &) { return std::make_shared<FunctionArrayIndex>(); } static FunctionPtr create(const Context &) { return std::make_shared<FunctionArrayIndex>(); }
private: private:
using ResultType = typename IndexConv::ResultType; using ResultType = typename ConcreteAction::ResultType;
using ResultColumnType = ColumnVector<ResultType>; using ResultColumnType = ColumnVector<ResultType>;
/** /**
@ -682,7 +682,7 @@ private:
* (s1, s1, s2, ...), (s2, s1, s2, ...), (s3, s1, s2, ...) * (s1, s1, s2, ...), (s2, s1, s2, ...), (s3, s1, s2, ...)
*/ */
template <class ...Integral> template <class ...Integral>
[[gnu::nonnull]] inline bool [[gnu::nonnull]] inline bool
executeIntegral(const ColumnArray * col, Block & block, const ColumnNumbers & arguments, size_t result) executeIntegral(const ColumnArray * col, Block & block, const ColumnNumbers & arguments, size_t result)
{ {
return (executeIntegralExpanded<Integral, Integral...>(col, block, arguments, result) || ...); return (executeIntegralExpanded<Integral, Integral...>(col, block, arguments, result) || ...);
@ -716,13 +716,10 @@ private:
const IColumn* item_arg = block.getByPosition(arguments[1]).column.get(); const IColumn* item_arg = block.getByPosition(arguments[1]).column.get();
if (item_arg->onlyNull()) if (item_arg->onlyNull())
ArrayIndexNumNullImpl<Initial, IndexConv>::vector( ArrayIndexNumNullImpl<Initial, ConcreteAction>::vector(
col_nested->getData(), col_nested->getData(), col_array->getOffsets(), col_res->getData(), null_map_data);
col_array->getOffsets(),
col_res->getData(),
null_map_data);
else if (const auto item_arg_const = checkAndGetColumnConst<ColumnVector<Resulting>>(item_arg)) else if (const auto item_arg_const = checkAndGetColumnConst<ColumnVector<Resulting>>(item_arg))
ArrayIndexNumImpl<Initial, Resulting, IndexConv>::vector( ArrayIndexNumImpl<Initial, Resulting, ConcreteAction>::vector(
col_nested->getData(), col_nested->getData(),
col_array->getOffsets(), col_array->getOffsets(),
item_arg_const->template getValue<Resulting>(), item_arg_const->template getValue<Resulting>(),
@ -730,7 +727,7 @@ private:
null_map_data, null_map_data,
nullptr); nullptr);
else if (const auto item_arg_vector = checkAndGetColumn<ColumnVector<Resulting>>(item_arg)) else if (const auto item_arg_vector = checkAndGetColumn<ColumnVector<Resulting>>(item_arg))
ArrayIndexNumImpl<Initial, Resulting, IndexConv>::vector( ArrayIndexNumImpl<Initial, Resulting, ConcreteAction>::vector(
col_nested->getData(), col_nested->getData(),
col_array->getOffsets(), col_array->getOffsets(),
item_arg_vector->getData(), item_arg_vector->getData(),
@ -772,18 +769,19 @@ private:
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
StringRef elem = col_arg->getDataAt(i); StringRef elem = col_arg->getDataAt(i);
UInt64 value_index = col_lc->getDictionary().getIndexByValue(elem); UInt64 value_index = col_lc->getDictionary().getOrFindIndex(elem);
ArrayIndexNumImpl< ArrayIndexNumImpl<
/* Initial data type -- DB::ReverseIndex index */ UInt64, /* Initial data type -- DB::ReverseIndex index */ UInt64,
/* Resulting data type -- same */ UInt64, /* Resulting data type -- same */ UInt64,
IndexConv>::vector( ConcreteAction>::
/* data -- indices column */ col_lc->getIndexes(), vector(
col_array->getOffsets(), /* data -- indices column */ col_lc->getIndexes(),
/* target value */ value_index, col_array->getOffsets(),
col_res->getData(), /* target value */ value_index,
null_map_data, col_res->getData(),
null_map_item); null_map_data,
null_map_item);
} }
block.getByPosition(result).column = std::move(col_res); block.getByPosition(result).column = std::move(col_res);
@ -805,23 +803,19 @@ private:
if (item_arg->onlyNull()) if (item_arg->onlyNull())
{ {
ArrayIndexStringNullImpl<IndexConv>::vector_const( ArrayIndexStringNullImpl<ConcreteAction>::vector_const(
col_nested->getChars(), col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), col_res->getData(), null_map_data);
col_array->getOffsets(),
col_nested->getOffsets(),
col_res->getData(),
null_map_data);
} }
else if (const auto item_arg_const = checkAndGetColumnConstStringOrFixedString(item_arg)) else if (const auto item_arg_const = checkAndGetColumnConstStringOrFixedString(item_arg))
{ {
const ColumnString * item_const_string = const ColumnString * item_const_string =
checkAndGetColumn<ColumnString>(&item_arg_const->getDataColumn()); checkAndGetColumn<ColumnString>(&item_arg_const->getDataColumn());
const ColumnFixedString * item_const_fixedstring = const ColumnFixedString * item_const_fixedstring =
checkAndGetColumn<ColumnFixedString>(&item_arg_const->getDataColumn()); checkAndGetColumn<ColumnFixedString>(&item_arg_const->getDataColumn());
if (item_const_string) if (item_const_string)
ArrayIndexStringImpl<IndexConv>::vector_const( ArrayIndexStringImpl<ConcreteAction>::vector_const(
col_nested->getChars(), col_nested->getChars(),
col_array->getOffsets(), col_array->getOffsets(),
col_nested->getOffsets(), col_nested->getOffsets(),
@ -830,7 +824,7 @@ private:
col_res->getData(), col_res->getData(),
null_map_data); null_map_data);
else if (item_const_fixedstring) else if (item_const_fixedstring)
ArrayIndexStringImpl<IndexConv>::vector_const( ArrayIndexStringImpl<ConcreteAction>::vector_const(
col_nested->getChars(), col_nested->getChars(),
col_array->getOffsets(), col_array->getOffsets(),
col_nested->getOffsets(), col_nested->getOffsets(),
@ -840,12 +834,12 @@ private:
null_map_data); null_map_data);
else else
throw Exception( throw Exception(
"Logical error: ColumnConst contains not String nor FixedString column", "Logical error: ColumnConst contains not String nor FixedString column",
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
} }
else if (const auto item_arg_vector = checkAndGetColumn<ColumnString>(item_arg)) else if (const auto item_arg_vector = checkAndGetColumn<ColumnString>(item_arg))
{ {
ArrayIndexStringImpl<IndexConv>::vectorVector( ArrayIndexStringImpl<ConcreteAction>::vectorVector(
col_nested->getChars(), col_nested->getChars(),
col_array->getOffsets(), col_array->getOffsets(),
col_nested->getOffsets(), col_nested->getOffsets(),
@ -862,10 +856,9 @@ private:
return true; return true;
} }
[[gnu::nonnull]]
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result) bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
{ {
const ColumnConst * col_array = const ColumnConst * col_array =
checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[0]).column.get()); checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[0]).column.get());
if (!col_array) if (!col_array)
@ -877,21 +870,20 @@ private:
if (isColumnConst(*item_arg)) if (isColumnConst(*item_arg))
{ {
typename IndexConv::ResultType current = 0; typename ConcreteAction::ResultType current = 0;
const auto & value = (*item_arg)[0]; const auto & value = (*item_arg)[0];
for (size_t i = 0, size = arr.size(); i < size; ++i) for (size_t i = 0, size = arr.size(); i < size; ++i)
{ {
if (applyVisitor(FieldVisitorAccurateEquals(), arr[i], value)) if (applyVisitor(FieldVisitorAccurateEquals(), arr[i], value))
{ {
if (!IndexConv::apply(i, current)) if (!ConcreteAction::apply(i, current))
break; break;
} }
} }
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst( block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(
item_arg->size(), item_arg->size(), static_cast<typename ConcreteAction::ResultType>(current));
static_cast<typename IndexConv::ResultType>(current));
} }
else else
{ {
@ -929,7 +921,7 @@ private:
if (hit) if (hit)
{ {
if (!IndexConv::apply(i, data[row])) if (!ConcreteAction::apply(i, data[row]))
break; break;
} }
} }
@ -952,25 +944,18 @@ private:
auto [null_map_data, null_map_item] = nullMapsBuilder(block, arguments); auto [null_map_data, null_map_item] = nullMapsBuilder(block, arguments);
if (item_arg.onlyNull()) if (item_arg.onlyNull())
ArrayIndexGenericNullImpl<IndexConv>::vector( ArrayIndexGenericNullImpl<ConcreteAction>::vector(col_nested, col->getOffsets(), col_res->getData(), null_map_data);
col_nested,
col->getOffsets(),
col_res->getData(),
null_map_data);
else if (isColumnConst(item_arg)) else if (isColumnConst(item_arg))
ArrayIndexGenericImpl<IndexConv, true>::vector( ArrayIndexGenericImpl<ConcreteAction, true>::vector(
col_nested, col_nested,
col->getOffsets(), col->getOffsets(),
assert_cast<const ColumnConst &>(item_arg).getDataColumn(), assert_cast<const ColumnConst &>(item_arg).getDataColumn(),
col_res->getData(), /// TODO This is wrong. col_res->getData(), /// TODO This is wrong.
null_map_data, nullptr); null_map_data,
nullptr);
else else
ArrayIndexGenericImpl<IndexConv, false>::vector( ArrayIndexGenericImpl<ConcreteAction, false>::vector(
col_nested, col_nested, col->getOffsets(), *item_arg.convertToFullColumnIfConst(), col_res->getData(), null_map_data, null_map_item);
col->getOffsets(),
*item_arg.convertToFullColumnIfConst(),
col_res->getData(),
null_map_data, null_map_item);
block.getByPosition(result).column = std::move(col_res); block.getByPosition(result).column = std::move(col_res);
return true; return true;
@ -995,7 +980,7 @@ public:
throw Exception("First argument for function " + getName() + " must be an array.", throw Exception("First argument for function " + getName() + " must be an array.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!arguments[1]->onlyNull() if (!arguments[1]->onlyNull()
&& !allowArrayIndex(array_type->getNestedType(), arguments[1]) && !allowArrayIndex(array_type->getNestedType(), arguments[1])
&& !allowNonLowCardinalityArg(array_type->getNestedType(), arguments[1])) && !allowNonLowCardinalityArg(array_type->getNestedType(), arguments[1]))
throw Exception("Types of array and 2nd argument of function " throw Exception("Types of array and 2nd argument of function "
@ -1003,7 +988,7 @@ public:
+ arguments[0]->getName() + " and " + arguments[1]->getName() + ".", + arguments[0]->getName() + " and " + arguments[1]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<typename IndexConv::ResultType>>(); return std::make_shared<DataTypeNumber<typename ConcreteAction::ResultType>>();
} }
/** /**