ClickHouse/src/Columns/ReverseIndex.h

533 lines
18 KiB
C++
Raw Normal View History

#pragma once
#include <Common/HashTable/Hash.h>
#include <Common/HashTable/HashTable.h>
#include <Common/HashTable/HashTableAllocator.h>
#include <Columns/ColumnString.h>
2018-08-28 19:36:27 +00:00
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
2021-10-02 07:13:14 +00:00
#include <base/range.h>
#include <base/unaligned.h>
2018-12-14 17:50:10 +00:00
namespace DB
{
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
template <typename ColumnType, bool with_saved_hash, bool has_base_index>
struct ReverseIndexHashTableState;
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ false>
{
constexpr static bool with_saved_hash = false;
constexpr static bool has_base_index = false;
ColumnType * index_column;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ true>
{
constexpr static bool with_saved_hash = false;
constexpr static bool has_base_index = true;
ColumnType * index_column;
size_t base_index;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ false>
{
constexpr static bool with_saved_hash = true;
constexpr static bool has_base_index = false;
ColumnType * index_column;
typename ColumnVector<UInt64>::Container * saved_hash_column;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ true>
{
constexpr static bool with_saved_hash = true;
constexpr static bool has_base_index = true;
ColumnType * index_column;
typename ColumnVector<UInt64>::Container * saved_hash_column;
size_t base_index;
};
2018-12-12 19:21:48 +00:00
struct ReverseIndexHash
{
template <typename T>
size_t operator()(T) const
{
throw Exception("operator()(key) is not implemented for ReverseIndexHash.", ErrorCodes::LOGICAL_ERROR);
}
};
template <typename IndexType, typename Hash, typename HashTable, typename ColumnType, bool string_hash, bool has_base_index>
2018-11-26 00:56:50 +00:00
struct ReverseIndexHashTableCell
: public HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>
{
using Base = HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>;
using State = typename Base::State;
using Base::Base;
using Base::key;
using Base::keyEquals;
using Base::isZero;
template <typename T>
static bool isZero(const T &, const State & /*state*/)
{
/// Careful: apparently this uses SFINAE to redefine isZero for all types
/// except the IndexType, for which the default ZeroTraits::isZero is used.
static_assert(!std::is_same_v<typename std::decay<T>::type, typename std::decay<IndexType>::type>);
return false;
}
2018-12-12 19:21:48 +00:00
/// Special case when we want to compare with something not in index_column.
/// When we compare something inside column default keyEquals checks only that row numbers are equal.
bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const
{
auto index = key;
if constexpr (has_base_index)
index -= state.base_index;
if constexpr (string_hash)
return hash_ == (*state.saved_hash_column)[index] && object == state.index_column->getDataAt(index);
else
return object == state.index_column->getDataAt(index);
}
size_t getHash(const Hash & hash) const
{
auto index = key;
/// Hack. HashTable is Hash itself.
const auto & state = static_cast<const State &>(static_cast<const HashTable &>(hash));
if constexpr (has_base_index)
index -= state.base_index;
if constexpr (string_hash)
return (*state.saved_hash_column)[index];
else
2018-12-12 19:21:48 +00:00
{
using ValueType = typename ColumnType::ValueType;
2018-12-14 17:50:10 +00:00
ValueType value = unalignedLoad<ValueType>(state.index_column->getDataAt(index).data);
2018-12-12 19:21:48 +00:00
return DefaultHash<ValueType>()(value);
}
}
};
/**
* ReverseIndexHashTableBase implements a special hash table interface for
* reverse index.
*
* The following requirements are different compared to a plain hash table:
*
* 1) Provide public access to 'hash table state' that contains
* additional data needed to calculate cell hashes.
*
* 2) Support emplace() and find() with a Key different from the resulting
* hash table key. This means emplace() accepts a different kind of object
* as a key, and then the real key can be read from the returned cell iterator.
*
* These requirements are unique to ReverseIndex and are in conflict with
* supporting hash tables that use alternative key storage, such as FixedHashMap
* or StringHashMap. Therefore, we implement an interface for ReverseIndex
* separately.
*/
template <typename Key, typename Cell, typename Hash>
class ReverseIndexHashTableBase : public HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>
{
using State = typename Cell::State;
using Base = HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>;
public:
using Base::Base;
using iterator = typename Base::iterator;
using LookupResult = typename Base::LookupResult;
State & getState() { return *this; }
template <typename ObjectToCompareWith>
size_t ALWAYS_INLINE reverseIndexFindCell(const ObjectToCompareWith & x,
size_t hash_value, size_t place_value) const
{
while (!this->buf[place_value].isZero(*this)
&& !this->buf[place_value].keyEquals(x, hash_value, *this))
{
place_value = this->grower.next(place_value);
}
return place_value;
}
template <typename ObjectToCompareWith>
void ALWAYS_INLINE reverseIndexEmplaceNonZero(const Key & key, LookupResult & it,
bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
{
size_t place_value = reverseIndexFindCell(object, hash_value,
this->grower.place(hash_value));
// emplaceNonZeroImpl() might need to re-find the cell if the table grows,
// but it will find it correctly by the key alone, so we don't have to
// pass it the 'object'.
this->emplaceNonZeroImpl(place_value, key, it, inserted, hash_value);
}
/// Searches position by object.
template <typename ObjectToCompareWith>
void ALWAYS_INLINE reverseIndexEmplace(Key key, iterator & it, bool & inserted,
size_t hash_value, const ObjectToCompareWith& object)
{
LookupResult impl_it = nullptr;
if (!this->emplaceIfZero(key, impl_it, inserted, hash_value))
{
reverseIndexEmplaceNonZero(key, impl_it, inserted, hash_value, object);
}
assert(impl_it != nullptr);
it = iterator(this, impl_it);
}
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE reverseIndexFind(ObjectToCompareWith x, size_t hash_value)
{
if (Cell::isZero(x, *this))
return this->hasZero() ? this->iteratorToZero() : this->end();
size_t place_value = reverseIndexFindCell(x, hash_value,
this->grower.place(hash_value));
return !this->buf[place_value].isZero(*this)
? iterator(this, &this->buf[place_value])
: this->end();
}
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexStringHashTable : public ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
2018-12-12 19:21:48 +00:00
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>,
2018-12-12 19:21:48 +00:00
ReverseIndexHash>
{
using Base = ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
2018-12-12 19:21:48 +00:00
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>,
2018-12-12 19:21:48 +00:00
ReverseIndexHash>;
public:
using Base::Base;
2018-08-09 09:28:52 +00:00
friend struct ReverseIndexHashTableCell<
IndexType,
2018-12-12 19:21:48 +00:00
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexNumberHashTable : public ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
2018-12-12 19:21:48 +00:00
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>,
2018-12-12 19:21:48 +00:00
ReverseIndexHash>
{
using Base = ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
2018-12-12 19:21:48 +00:00
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>,
2018-12-12 19:21:48 +00:00
ReverseIndexHash>;
public:
using Base::Base;
2018-08-08 16:44:53 +00:00
friend struct ReverseIndexHashTableCell<
IndexType,
2018-12-12 19:21:48 +00:00
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index, bool is_numeric_column>
struct SelectReverseIndexHashTable;
template <typename IndexType, typename ColumnType, bool has_base_index>
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, true>
{
using Type = ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index>
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, false>
{
using Type = ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>;
};
template <typename T>
constexpr bool isNumericColumn(const T *) { return false; }
template <typename T>
constexpr bool isNumericColumn(const ColumnVector<T> *) { return true; }
static_assert(isNumericColumn(static_cast<ColumnVector<UInt8> *>(nullptr)));
static_assert(!isNumericColumn(static_cast<ColumnString *>(nullptr)));
template <typename IndexType, typename ColumnType, bool has_base_index>
using ReverseIndexHashTable = typename SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index,
isNumericColumn(static_cast<ColumnType *>(nullptr))>::Type;
}
template <typename IndexType, typename ColumnType>
class ReverseIndex
{
public:
2021-02-17 23:11:41 +00:00
ReverseIndex(UInt64 num_prefix_rows_to_skip_, UInt64 base_index_)
Fix concurrent access to LowCardinality during GROUP BY (leads to SIGSEGV) The problem is that GROUP BY can update saved_hash, which can be also updated by subsequent update of a dictionary, and this will lead to use-after-free. You will find ASan report in `details`. <details> ==24679==ERROR: AddressSanitizer: heap-use-after-free on address 0x604000615d20 at pc 0x000022cc8684 bp 0x7ffea6b5f850 sp 0x7ffea6b5f848 READ of size 8 at 0x604000615d20 thread T223 (QueryPipelineEx) 0 0x22cc8683 in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h 1 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39 2 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23 3 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26 4 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28 5 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62 6 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23 7 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9 8 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9 9 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 0x604000615d20 is located 16 bytes inside of 40-byte region [0x604000615d10,0x604000615d38) freed by thread T37 (QueryPipelineEx) here: 2 0x22cb9392 in boost::intrusive_ptr<DB::ColumnVector<unsigned long> >::~intrusive_ptr() obj-x86_64-linux-gnu/../contrib/boost/boost/smart_ptr/intrusive_ptr.hpp:98:23 4 0x22cb9392 in COW<DB::IColumn>::mutable_ptr<DB::ColumnVector<unsigned long> >::operator=() obj-x86_64-linux-gnu/../src/Common/COW.h:100:57 5 0x22cb9392 in DB::ReverseIndex<>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:362:28 6 0x22cb9392 in DB::ColumnUnique<DB::ColumnString>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:125:76 7 0x242eaed3 in DB::ColumnsHashing::HashMethodSingleLowCardinalityColumn<>::HashMethodSingleLowCardinalityColumn() obj-x86_64-linux-gnu/../src/Common/ColumnsHashing.h:287:50 8 0x242206c6 in void DB::Aggregator::executeImpl<>() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:596:28 9 0x24148e99 in DB::Aggregator::executeOnBlock() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:1004:9 10 0x26c24f3a in DB::AggregatingTransform::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:539:33 11 0x26c2054e in DB::AggregatingTransform::work() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:500:9 previously allocated by thread T37 (QueryPipelineEx) here: 0 0xb6d44fd in operator new(unsigned long) (/src/ch/tmp/upstream/clickhouse-asan+0xb6d44fd) 1 0x11b78580 in COW<DB::IColumn>::mutable_ptr<> COWHelper<>::create<unsigned long&>(unsigned long&) (/src/ch/tmp/upstream/clickhouse-asan+0x11b78580) 2 0x22cbf7b1 in DB::ReverseIndex<>::calcHashes() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:472:17 3 0x22cc2307 in DB::ReverseIndex<>::buildIndex() obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:438:22 4 0x22cc658c in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:484:9 5 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39 6 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23 7 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26 8 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28 9 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62 10 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23 11 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9 12 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9 13 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 SUMMARY: AddressSanitizer: heap-use-after-free obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h in DB::ReverseIndex<unsigned long, DB::ColumnString>::insert(StringRef const&) Shadow bytes around the buggy address: 0x0c08800bab50: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 00 0x0c08800bab60: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 fa 0x0c08800bab70: fa fa 00 00 00 00 00 00 fa fa 00 00 00 00 00 00 0x0c08800bab80: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fa 0x0c08800bab90: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd =>0x0c08800baba0: fa fa fd fd[fd]fd fd fa fa fa fd fd fd fd fd fa 0x0c08800babb0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa 0x0c08800babc0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd 0x0c08800babd0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa 0x0c08800babe0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd 0x0c08800babf0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fd Shadow byte legend (one shadow byte represents 8 application bytes): Addressable: 00 Partially addressable: 01 02 03 04 05 06 07 Heap left redzone: fa Freed heap region: fd Stack left redzone: f1 Stack mid redzone: f2 Stack right redzone: f3 Stack after return: f5 Stack use after scope: f8 Global redzone: f9 Global init order: f6 Poisoned by user: f7 Container overflow: fc Array cookie: ac Intra object redzone: bb ASan internal: fe Left alloca redzone: ca Right alloca redzone: cb ==24679==ABORTING </details>
2021-10-05 18:06:36 +00:00
: num_prefix_rows_to_skip(num_prefix_rows_to_skip_), base_index(base_index_), external_saved_hash_ptr(nullptr) {}
void setColumn(ColumnType * column_);
static constexpr bool is_numeric_column = isNumericColumn(static_cast<ColumnType *>(nullptr));
static constexpr bool use_saved_hash = !is_numeric_column;
2018-12-12 19:21:48 +00:00
UInt64 insert(const StringRef & data);
2020-07-16 15:08:51 +00:00
2020-07-23 16:46:38 +00:00
/// Returns the found data's index in the dictionary. If index is not built, builds it.
UInt64 getInsertionPoint(StringRef data)
{
2021-02-17 23:11:41 +00:00
if (!index)
buildIndex();
2020-07-23 16:46:38 +00:00
return getIndexImpl(data);
}
2020-07-16 15:08:51 +00:00
2020-07-22 13:18:21 +00:00
/// Returns the found data's index in the dictionary if the #index is built, otherwise, returns a std::nullopt.
2020-07-23 16:46:38 +00:00
std::optional<UInt64> getIndex(StringRef data) const
{
2021-02-17 23:11:41 +00:00
if (!index)
return {};
2020-07-23 16:46:38 +00:00
return getIndexImpl(data);
}
2020-07-16 15:08:51 +00:00
UInt64 lastInsertionPoint() const { return size() + base_index; }
ColumnType * getColumn() const { return column; }
size_t size() const;
const UInt64 * tryGetSavedHash() const
{
if (!use_saved_hash)
return nullptr;
Fix concurrent access to LowCardinality during GROUP BY (leads to SIGSEGV) The problem is that GROUP BY can update saved_hash, which can be also updated by subsequent update of a dictionary, and this will lead to use-after-free. You will find ASan report in `details`. <details> ==24679==ERROR: AddressSanitizer: heap-use-after-free on address 0x604000615d20 at pc 0x000022cc8684 bp 0x7ffea6b5f850 sp 0x7ffea6b5f848 READ of size 8 at 0x604000615d20 thread T223 (QueryPipelineEx) 0 0x22cc8683 in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h 1 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39 2 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23 3 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26 4 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28 5 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62 6 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23 7 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9 8 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9 9 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 0x604000615d20 is located 16 bytes inside of 40-byte region [0x604000615d10,0x604000615d38) freed by thread T37 (QueryPipelineEx) here: 2 0x22cb9392 in boost::intrusive_ptr<DB::ColumnVector<unsigned long> >::~intrusive_ptr() obj-x86_64-linux-gnu/../contrib/boost/boost/smart_ptr/intrusive_ptr.hpp:98:23 4 0x22cb9392 in COW<DB::IColumn>::mutable_ptr<DB::ColumnVector<unsigned long> >::operator=() obj-x86_64-linux-gnu/../src/Common/COW.h:100:57 5 0x22cb9392 in DB::ReverseIndex<>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:362:28 6 0x22cb9392 in DB::ColumnUnique<DB::ColumnString>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:125:76 7 0x242eaed3 in DB::ColumnsHashing::HashMethodSingleLowCardinalityColumn<>::HashMethodSingleLowCardinalityColumn() obj-x86_64-linux-gnu/../src/Common/ColumnsHashing.h:287:50 8 0x242206c6 in void DB::Aggregator::executeImpl<>() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:596:28 9 0x24148e99 in DB::Aggregator::executeOnBlock() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:1004:9 10 0x26c24f3a in DB::AggregatingTransform::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:539:33 11 0x26c2054e in DB::AggregatingTransform::work() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:500:9 previously allocated by thread T37 (QueryPipelineEx) here: 0 0xb6d44fd in operator new(unsigned long) (/src/ch/tmp/upstream/clickhouse-asan+0xb6d44fd) 1 0x11b78580 in COW<DB::IColumn>::mutable_ptr<> COWHelper<>::create<unsigned long&>(unsigned long&) (/src/ch/tmp/upstream/clickhouse-asan+0x11b78580) 2 0x22cbf7b1 in DB::ReverseIndex<>::calcHashes() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:472:17 3 0x22cc2307 in DB::ReverseIndex<>::buildIndex() obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:438:22 4 0x22cc658c in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:484:9 5 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39 6 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23 7 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26 8 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28 9 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62 10 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23 11 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9 12 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9 13 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 SUMMARY: AddressSanitizer: heap-use-after-free obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h in DB::ReverseIndex<unsigned long, DB::ColumnString>::insert(StringRef const&) Shadow bytes around the buggy address: 0x0c08800bab50: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 00 0x0c08800bab60: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 fa 0x0c08800bab70: fa fa 00 00 00 00 00 00 fa fa 00 00 00 00 00 00 0x0c08800bab80: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fa 0x0c08800bab90: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd =>0x0c08800baba0: fa fa fd fd[fd]fd fd fa fa fa fd fd fd fd fd fa 0x0c08800babb0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa 0x0c08800babc0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd 0x0c08800babd0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa 0x0c08800babe0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd 0x0c08800babf0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fd Shadow byte legend (one shadow byte represents 8 application bytes): Addressable: 00 Partially addressable: 01 02 03 04 05 06 07 Heap left redzone: fa Freed heap region: fd Stack left redzone: f1 Stack mid redzone: f2 Stack right redzone: f3 Stack after return: f5 Stack use after scope: f8 Global redzone: f9 Global init order: f6 Poisoned by user: f7 Container overflow: fc Array cookie: ac Intra object redzone: bb ASan internal: fe Left alloca redzone: ca Right alloca redzone: cb ==24679==ABORTING </details>
2021-10-05 18:06:36 +00:00
UInt64 * ptr = external_saved_hash_ptr.load();
if (!ptr)
{
auto hash = calcHashes();
ptr = &hash->getData()[0];
UInt64 * expected = nullptr;
Fix concurrent access to LowCardinality during GROUP BY (leads to SIGSEGV) The problem is that GROUP BY can update saved_hash, which can be also updated by subsequent update of a dictionary, and this will lead to use-after-free. You will find ASan report in `details`. <details> ==24679==ERROR: AddressSanitizer: heap-use-after-free on address 0x604000615d20 at pc 0x000022cc8684 bp 0x7ffea6b5f850 sp 0x7ffea6b5f848 READ of size 8 at 0x604000615d20 thread T223 (QueryPipelineEx) 0 0x22cc8683 in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h 1 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39 2 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23 3 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26 4 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28 5 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62 6 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23 7 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9 8 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9 9 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 0x604000615d20 is located 16 bytes inside of 40-byte region [0x604000615d10,0x604000615d38) freed by thread T37 (QueryPipelineEx) here: 2 0x22cb9392 in boost::intrusive_ptr<DB::ColumnVector<unsigned long> >::~intrusive_ptr() obj-x86_64-linux-gnu/../contrib/boost/boost/smart_ptr/intrusive_ptr.hpp:98:23 4 0x22cb9392 in COW<DB::IColumn>::mutable_ptr<DB::ColumnVector<unsigned long> >::operator=() obj-x86_64-linux-gnu/../src/Common/COW.h:100:57 5 0x22cb9392 in DB::ReverseIndex<>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:362:28 6 0x22cb9392 in DB::ColumnUnique<DB::ColumnString>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:125:76 7 0x242eaed3 in DB::ColumnsHashing::HashMethodSingleLowCardinalityColumn<>::HashMethodSingleLowCardinalityColumn() obj-x86_64-linux-gnu/../src/Common/ColumnsHashing.h:287:50 8 0x242206c6 in void DB::Aggregator::executeImpl<>() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:596:28 9 0x24148e99 in DB::Aggregator::executeOnBlock() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:1004:9 10 0x26c24f3a in DB::AggregatingTransform::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:539:33 11 0x26c2054e in DB::AggregatingTransform::work() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:500:9 previously allocated by thread T37 (QueryPipelineEx) here: 0 0xb6d44fd in operator new(unsigned long) (/src/ch/tmp/upstream/clickhouse-asan+0xb6d44fd) 1 0x11b78580 in COW<DB::IColumn>::mutable_ptr<> COWHelper<>::create<unsigned long&>(unsigned long&) (/src/ch/tmp/upstream/clickhouse-asan+0x11b78580) 2 0x22cbf7b1 in DB::ReverseIndex<>::calcHashes() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:472:17 3 0x22cc2307 in DB::ReverseIndex<>::buildIndex() obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:438:22 4 0x22cc658c in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:484:9 5 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39 6 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23 7 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26 8 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28 9 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62 10 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23 11 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9 12 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9 13 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 SUMMARY: AddressSanitizer: heap-use-after-free obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h in DB::ReverseIndex<unsigned long, DB::ColumnString>::insert(StringRef const&) Shadow bytes around the buggy address: 0x0c08800bab50: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 00 0x0c08800bab60: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 fa 0x0c08800bab70: fa fa 00 00 00 00 00 00 fa fa 00 00 00 00 00 00 0x0c08800bab80: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fa 0x0c08800bab90: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd =>0x0c08800baba0: fa fa fd fd[fd]fd fd fa fa fa fd fd fd fd fd fa 0x0c08800babb0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa 0x0c08800babc0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd 0x0c08800babd0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa 0x0c08800babe0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd 0x0c08800babf0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fd Shadow byte legend (one shadow byte represents 8 application bytes): Addressable: 00 Partially addressable: 01 02 03 04 05 06 07 Heap left redzone: fa Freed heap region: fd Stack left redzone: f1 Stack mid redzone: f2 Stack right redzone: f3 Stack after return: f5 Stack use after scope: f8 Global redzone: f9 Global init order: f6 Poisoned by user: f7 Container overflow: fc Array cookie: ac Intra object redzone: bb ASan internal: fe Left alloca redzone: ca Right alloca redzone: cb ==24679==ABORTING </details>
2021-10-05 18:06:36 +00:00
if (external_saved_hash_ptr.compare_exchange_strong(expected, ptr))
external_saved_hash = std::move(hash);
else
ptr = expected;
}
return ptr;
}
size_t allocatedBytes() const { return index ? index->getBufferSizeInBytes() : 0; }
private:
ColumnType * column = nullptr;
UInt64 num_prefix_rows_to_skip; /// The number prefix tows in column which won't be sored at index.
UInt64 base_index; /// This values will be added to row number which is inserted into index.
using IndexMapType = ReverseIndexHashTable<IndexType, ColumnType, true>;
/// Lazy initialized.
std::unique_ptr<IndexMapType> index;
mutable ColumnUInt64::MutablePtr saved_hash;
Fix concurrent access to LowCardinality during GROUP BY (leads to SIGSEGV) The problem is that GROUP BY can update saved_hash, which can be also updated by subsequent update of a dictionary, and this will lead to use-after-free. You will find ASan report in `details`. <details> ==24679==ERROR: AddressSanitizer: heap-use-after-free on address 0x604000615d20 at pc 0x000022cc8684 bp 0x7ffea6b5f850 sp 0x7ffea6b5f848 READ of size 8 at 0x604000615d20 thread T223 (QueryPipelineEx) 0 0x22cc8683 in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h 1 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39 2 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23 3 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26 4 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28 5 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62 6 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23 7 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9 8 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9 9 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 0x604000615d20 is located 16 bytes inside of 40-byte region [0x604000615d10,0x604000615d38) freed by thread T37 (QueryPipelineEx) here: 2 0x22cb9392 in boost::intrusive_ptr<DB::ColumnVector<unsigned long> >::~intrusive_ptr() obj-x86_64-linux-gnu/../contrib/boost/boost/smart_ptr/intrusive_ptr.hpp:98:23 4 0x22cb9392 in COW<DB::IColumn>::mutable_ptr<DB::ColumnVector<unsigned long> >::operator=() obj-x86_64-linux-gnu/../src/Common/COW.h:100:57 5 0x22cb9392 in DB::ReverseIndex<>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:362:28 6 0x22cb9392 in DB::ColumnUnique<DB::ColumnString>::tryGetSavedHash() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:125:76 7 0x242eaed3 in DB::ColumnsHashing::HashMethodSingleLowCardinalityColumn<>::HashMethodSingleLowCardinalityColumn() obj-x86_64-linux-gnu/../src/Common/ColumnsHashing.h:287:50 8 0x242206c6 in void DB::Aggregator::executeImpl<>() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:596:28 9 0x24148e99 in DB::Aggregator::executeOnBlock() const obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:1004:9 10 0x26c24f3a in DB::AggregatingTransform::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:539:33 11 0x26c2054e in DB::AggregatingTransform::work() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingTransform.cpp:500:9 previously allocated by thread T37 (QueryPipelineEx) here: 0 0xb6d44fd in operator new(unsigned long) (/src/ch/tmp/upstream/clickhouse-asan+0xb6d44fd) 1 0x11b78580 in COW<DB::IColumn>::mutable_ptr<> COWHelper<>::create<unsigned long&>(unsigned long&) (/src/ch/tmp/upstream/clickhouse-asan+0x11b78580) 2 0x22cbf7b1 in DB::ReverseIndex<>::calcHashes() const obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:472:17 3 0x22cc2307 in DB::ReverseIndex<>::buildIndex() obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:438:22 4 0x22cc658c in DB::ReverseIndex<>::insert(StringRef const&) obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h:484:9 5 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>()::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:540:39 6 0x22cc0de1 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeImpl<char8_t>() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:570:23 7 0x22cb9c66 in COW<DB::IColumn>::mutable_ptr<DB::IColumn> DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom()::'lambda'(auto)::operator()<char8_t>(auto) const obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:592:26 8 0x22cb9c66 in DB::ColumnUnique<DB::ColumnString>::uniqueInsertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnUnique.h:600:28 9 0x2500b897 in DB::ColumnLowCardinality::insertRangeFrom() obj-x86_64-linux-gnu/../src/Columns/ColumnLowCardinality.cpp:205:62 10 0x25a182f4 in DB::appendBlock(DB::Block const&, DB::Block&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:470:23 11 0x25a182f4 in DB::BufferSink::insertIntoBuffer(DB::Block const&, DB::StorageBuffer::Buffer&) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:634:9 12 0x25a173cc in DB::BufferSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:595:9 13 0x26d1c997 in DB::SinkToStorage::transform(DB::Chunk&) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 SUMMARY: AddressSanitizer: heap-use-after-free obj-x86_64-linux-gnu/../src/Columns/ReverseIndex.h in DB::ReverseIndex<unsigned long, DB::ColumnString>::insert(StringRef const&) Shadow bytes around the buggy address: 0x0c08800bab50: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 00 0x0c08800bab60: fa fa fd fd fd fd fd fa fa fa 00 00 00 00 00 fa 0x0c08800bab70: fa fa 00 00 00 00 00 00 fa fa 00 00 00 00 00 00 0x0c08800bab80: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fa 0x0c08800bab90: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd =>0x0c08800baba0: fa fa fd fd[fd]fd fd fa fa fa fd fd fd fd fd fa 0x0c08800babb0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa 0x0c08800babc0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd 0x0c08800babd0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fa 0x0c08800babe0: fa fa fd fd fd fd fd fa fa fa fd fd fd fd fd fd 0x0c08800babf0: fa fa fd fd fd fd fd fd fa fa fd fd fd fd fd fd Shadow byte legend (one shadow byte represents 8 application bytes): Addressable: 00 Partially addressable: 01 02 03 04 05 06 07 Heap left redzone: fa Freed heap region: fd Stack left redzone: f1 Stack mid redzone: f2 Stack right redzone: f3 Stack after return: f5 Stack use after scope: f8 Global redzone: f9 Global init order: f6 Poisoned by user: f7 Container overflow: fc Array cookie: ac Intra object redzone: bb ASan internal: fe Left alloca redzone: ca Right alloca redzone: cb ==24679==ABORTING </details>
2021-10-05 18:06:36 +00:00
/// For usage during GROUP BY
mutable ColumnUInt64::MutablePtr external_saved_hash;
mutable std::atomic<UInt64 *> external_saved_hash_ptr;
void buildIndex();
UInt64 getHash(const StringRef & ref) const
{
if constexpr (is_numeric_column)
{
using ValueType = typename ColumnType::ValueType;
2018-12-14 17:50:10 +00:00
ValueType value = unalignedLoad<ValueType>(ref.data);
return DefaultHash<ValueType>()(value);
}
else
return StringRefHash()(ref);
}
ColumnUInt64::MutablePtr calcHashes() const;
2020-07-23 16:46:38 +00:00
UInt64 getIndexImpl(StringRef data) const;
};
template <typename IndexType, typename ColumnType>
void ReverseIndex<IndexType, ColumnType>:: setColumn(ColumnType * column_)
{
if (column != column_)
{
index = nullptr;
saved_hash = nullptr;
}
column = column_;
}
template <typename IndexType, typename ColumnType>
size_t ReverseIndex<IndexType, ColumnType>::size() const
{
if (!column)
throw Exception("ReverseIndex has not size because index column wasn't set.", ErrorCodes::LOGICAL_ERROR);
return column->size();
}
template <typename IndexType, typename ColumnType>
void ReverseIndex<IndexType, ColumnType>::buildIndex()
{
if (index)
return;
if (!column)
throw Exception("ReverseIndex can't build index because index column wasn't set.", ErrorCodes::LOGICAL_ERROR);
auto size = column->size();
index = std::make_unique<IndexMapType>(size);
if constexpr (use_saved_hash)
saved_hash = calcHashes();
auto & state = index->getState();
state.index_column = column;
state.base_index = base_index;
if constexpr (use_saved_hash)
state.saved_hash_column = &saved_hash->getData();
using IteratorType = typename IndexMapType::iterator;
IteratorType iterator;
bool inserted;
2021-06-15 19:55:21 +00:00
for (auto row : collections::range(num_prefix_rows_to_skip, size))
{
UInt64 hash;
if constexpr (use_saved_hash)
hash = saved_hash->getElement(row);
else
hash = getHash(column->getDataAt(row));
index->reverseIndexEmplace(row + base_index, iterator, inserted, hash, column->getDataAt(row));
if (!inserted)
throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR);
}
}
template <typename IndexType, typename ColumnType>
ColumnUInt64::MutablePtr ReverseIndex<IndexType, ColumnType>::calcHashes() const
{
if (!column)
throw Exception("ReverseIndex can't build index because index column wasn't set.", ErrorCodes::LOGICAL_ERROR);
auto size = column->size();
auto hash = ColumnUInt64::create(size);
2021-06-15 19:55:21 +00:00
for (auto row : collections::range(0, size))
hash->getElement(row) = getHash(column->getDataAt(row));
2019-05-31 08:55:01 +00:00
return hash;
}
template <typename IndexType, typename ColumnType>
2018-12-12 19:21:48 +00:00
UInt64 ReverseIndex<IndexType, ColumnType>::insert(const StringRef & data)
{
if (!index)
buildIndex();
using IteratorType = typename IndexMapType::iterator;
IteratorType iterator;
bool inserted;
2018-12-12 19:21:48 +00:00
auto hash = getHash(data);
UInt64 num_rows = size();
if constexpr (use_saved_hash)
{
auto & column_data = saved_hash->getData();
if (column_data.size() <= num_rows)
column_data.resize(num_rows + 1);
column_data[num_rows] = hash;
}
2018-12-12 19:21:48 +00:00
else
column->insertData(data.data, data.size);
index->reverseIndexEmplace(num_rows + base_index, iterator, inserted, hash, data);
2018-12-12 19:21:48 +00:00
if constexpr (use_saved_hash)
{
if (inserted)
column->insertData(data.data, data.size);
}
else
{
if (!inserted)
column->popBack(1);
}
A Proper lookup table that uses HashTable's API This is the first step of allowing heterogeneous cells in hash tables. performance test results are ``` 1. HashMap<UInt16, UInt8, TrivialHash, HashTableFixedGrower<16>>; 2. NewLookupMap<UInt16, UInt8> ResolutionWidth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................223550276.46 ResolutionWidth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................248772721.24 Best: 2 - 24877272124 ResolutionWidth 100000 1 ..........................................................................................................................................................................................................................................................238498413.99 ResolutionWidth 100000 2 ..........................................................................................................................................................................................................................................................261808889.98 Best: 2 - 26180888998 ResolutionWidth 300000 1 ...................................................................................239307348.81 ResolutionWidth 300000 2 ...................................................................................257592761.30 Best: 2 - 25759276130 ResolutionWidth 1000000 1 .........................240144759.26 ResolutionWidth 1000000 2 .........................257093531.91 Best: 2 - 25709353191 ResolutionWidth 5000000 1 .....241573260.35 ResolutionWidth 5000000 2 .....259314162.79 Best: 2 - 25931416279 ResolutionDepth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................217108119.84 ResolutionDepth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................249459504.41 Best: 2 - 24945950441 ResolutionDepth 100000 1 ..........................................................................................................................................................................................................................................................229065162.17 ResolutionDepth 100000 2 ..........................................................................................................................................................................................................................................................253769105.64 Best: 2 - 25376910564 ResolutionDepth 300000 1 ...................................................................................233079225.18 ResolutionDepth 300000 2 ...................................................................................256316273.78 Best: 2 - 25631627378 ResolutionDepth 1000000 1 .........................234184633.51 ResolutionDepth 1000000 2 .........................261100491.57 Best: 2 - 26110049157 ResolutionDepth 5000000 1 .....233118795.66 ResolutionDepth 5000000 2 .....252436160.41 Best: 2 - 25243616041 ```
2019-02-28 09:35:38 +00:00
return iterator->getValue();
}
template <typename IndexType, typename ColumnType>
2020-07-23 16:46:38 +00:00
UInt64 ReverseIndex<IndexType, ColumnType>::getIndexImpl(StringRef data) const
{
using IteratorType = typename IndexMapType::iterator;
IteratorType iterator;
auto hash = getHash(data);
iterator = index->reverseIndexFind(data, hash);
A Proper lookup table that uses HashTable's API This is the first step of allowing heterogeneous cells in hash tables. performance test results are ``` 1. HashMap<UInt16, UInt8, TrivialHash, HashTableFixedGrower<16>>; 2. NewLookupMap<UInt16, UInt8> ResolutionWidth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................223550276.46 ResolutionWidth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................248772721.24 Best: 2 - 24877272124 ResolutionWidth 100000 1 ..........................................................................................................................................................................................................................................................238498413.99 ResolutionWidth 100000 2 ..........................................................................................................................................................................................................................................................261808889.98 Best: 2 - 26180888998 ResolutionWidth 300000 1 ...................................................................................239307348.81 ResolutionWidth 300000 2 ...................................................................................257592761.30 Best: 2 - 25759276130 ResolutionWidth 1000000 1 .........................240144759.26 ResolutionWidth 1000000 2 .........................257093531.91 Best: 2 - 25709353191 ResolutionWidth 5000000 1 .....241573260.35 ResolutionWidth 5000000 2 .....259314162.79 Best: 2 - 25931416279 ResolutionDepth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................217108119.84 ResolutionDepth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................249459504.41 Best: 2 - 24945950441 ResolutionDepth 100000 1 ..........................................................................................................................................................................................................................................................229065162.17 ResolutionDepth 100000 2 ..........................................................................................................................................................................................................................................................253769105.64 Best: 2 - 25376910564 ResolutionDepth 300000 1 ...................................................................................233079225.18 ResolutionDepth 300000 2 ...................................................................................256316273.78 Best: 2 - 25631627378 ResolutionDepth 1000000 1 .........................234184633.51 ResolutionDepth 1000000 2 .........................261100491.57 Best: 2 - 26110049157 ResolutionDepth 5000000 1 .....233118795.66 ResolutionDepth 5000000 2 .....252436160.41 Best: 2 - 25243616041 ```
2019-02-28 09:35:38 +00:00
return iterator == index->end() ? size() + base_index : iterator->getValue();
}
}