ClickHouse/src/Interpreters/RowRefs.h

300 lines
8.7 KiB
C++
Raw Normal View History

2019-03-30 21:30:21 +00:00
#pragma once
#include <algorithm>
#include <cassert>
2019-04-02 18:50:35 +00:00
#include <list>
#include <mutex>
#include <optional>
#include <variant>
2022-01-30 19:49:48 +00:00
2022-02-04 17:02:41 +00:00
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnVector.h>
#include <Columns/IColumn.h>
#include <Interpreters/asof.h>
#include <base/sort.h>
#include <Common/Arena.h>
2022-01-30 19:49:48 +00:00
2019-03-30 21:30:21 +00:00
namespace DB
{
class Block;
/// Reference to the row in block.
2022-02-19 10:16:28 +00:00
struct RowRef
2019-03-30 21:30:21 +00:00
{
2020-04-21 19:01:34 +00:00
using SizeT = uint32_t; /// Do not use size_t cause of memory economy
2019-03-30 21:30:21 +00:00
const Block * block = nullptr;
2020-04-21 19:01:34 +00:00
SizeT row_num = 0;
2019-03-30 21:30:21 +00:00
2022-02-18 10:02:14 +00:00
RowRef() = default;
2019-03-30 21:30:21 +00:00
RowRef(const Block * block_, size_t row_num_) : block(block_), row_num(row_num_) {}
};
/// Single linked list of references to rows. Used for ALL JOINs (non-unique JOINs)
struct RowRefList : RowRef
{
2019-05-14 14:40:43 +00:00
/// Portion of RowRefs, 16 * (MAX_SIZE + 1) bytes sized.
struct Batch
{
static constexpr size_t MAX_SIZE = 7; /// Adequate values are 3, 7, 15, 31.
2020-04-21 19:01:34 +00:00
SizeT size = 0; /// It's smaller than size_t but keeps align in Arena.
2019-05-14 14:40:43 +00:00
Batch * next;
RowRef row_refs[MAX_SIZE];
Batch(Batch * parent)
: next(parent)
{}
bool full() const { return size == MAX_SIZE; }
Batch * insert(RowRef && row_ref, Arena & pool)
{
if (full())
{
auto batch = pool.alloc<Batch>();
*batch = Batch(this);
batch->insert(std::move(row_ref), pool);
return batch;
}
row_refs[size++] = std::move(row_ref);
return this;
}
};
2019-05-14 14:39:03 +00:00
class ForwardIterator
{
public:
ForwardIterator(const RowRefList * begin)
2019-05-14 14:40:43 +00:00
: root(begin)
, first(true)
, batch(root->next)
, position(0)
2019-05-14 14:39:03 +00:00
{}
2019-05-14 14:40:43 +00:00
const RowRef * operator -> () const
{
if (first)
return root;
return &batch->row_refs[position];
}
const RowRef * operator * () const
{
if (first)
return root;
return &batch->row_refs[position];
}
2019-05-14 14:40:43 +00:00
void operator ++ ()
{
if (first)
{
first = false;
return;
}
if (batch)
{
++position;
if (position >= batch->size)
{
batch = batch->next;
position = 0;
}
}
}
bool ok() const { return first || batch; }
2019-05-14 14:39:03 +00:00
private:
2019-05-14 14:40:43 +00:00
const RowRefList * root;
bool first;
Batch * batch;
size_t position;
2019-05-14 14:39:03 +00:00
};
2019-03-30 21:30:21 +00:00
RowRefList() {}
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {}
2019-05-14 14:39:03 +00:00
ForwardIterator begin() const { return ForwardIterator(this); }
/// insert element after current one
void insert(RowRef && row_ref, Arena & pool)
{
2019-05-14 14:40:43 +00:00
if (!next)
{
next = pool.alloc<Batch>();
*next = Batch(nullptr);
}
next = next->insert(std::move(row_ref), pool);
2019-05-14 14:39:03 +00:00
}
private:
2019-05-14 14:40:43 +00:00
Batch * next = nullptr;
2019-03-30 21:30:21 +00:00
};
/**
* This class is intended to push sortable data into.
* When looking up values the container ensures that it is sorted for log(N) lookup
* After calling any of the lookup methods, it is no longer allowed to insert more data as this would invalidate the
* references that can be returned by the lookup methods
*/
struct SortedLookupVectorBase
{
SortedLookupVectorBase() = default;
virtual ~SortedLookupVectorBase() { }
static std::optional<TypeIndex> getTypeSize(const IColumn & asof_column, size_t & type_size);
// This will be synchronized by the rwlock mutex in Join.h
virtual void insert(const IColumn &, const Block *, size_t) = 0;
// This needs to be synchronized internally
2022-02-19 20:01:47 +00:00
virtual std::tuple<decltype(RowRef::block), decltype(RowRef::row_num)> findAsof(const IColumn &, size_t) = 0;
};
2022-02-18 15:16:29 +00:00
template <typename TKey, ASOF::Inequality inequality>
class SortedLookupVector : public SortedLookupVectorBase
{
2022-02-19 20:01:47 +00:00
public:
struct Entry
{
2022-02-19 20:01:47 +00:00
/// We don't store a RowRef and instead keep it's members separately (and return a tuple) to reduce the memory usage.
/// For example, for sizeof(T) == 4 => sizeof(Entry) == 16 (while before it would be 20). Then when you put it into a vector, the effect is even greater
decltype(RowRef::block) block;
decltype(RowRef::row_num) row_num;
TKey asof_value;
Entry() = delete;
2022-02-19 20:01:47 +00:00
Entry(TKey v, const Block * b, size_t r) : block(b), row_num(r), asof_value(v) { }
bool operator<(const Entry & other) const { return asof_value < other.asof_value; }
};
2022-02-18 15:50:15 +00:00
struct greaterEntryOperator
{
bool operator()(Entry const & a, Entry const & b) const { return a.asof_value > b.asof_value; }
};
public:
using Base = std::vector<Entry>;
2022-02-04 16:12:01 +00:00
using Keys = std::vector<TKey>;
2022-02-18 15:50:15 +00:00
static constexpr bool isDescending = (inequality == ASOF::Inequality::Greater || inequality == ASOF::Inequality::GreaterOrEquals);
static constexpr bool isStrict = (inequality == ASOF::Inequality::Less) || (inequality == ASOF::Inequality::Greater);
void insert(const IColumn & asof_column, const Block * block, size_t row_num) override
{
using ColumnType = ColumnVectorOrDecimal<TKey>;
const auto & column = assert_cast<const ColumnType &>(asof_column);
TKey k = column.getElement(row_num);
assert(!sorted.load(std::memory_order_acquire));
array.emplace_back(k, block, row_num);
}
2022-02-18 15:16:29 +00:00
/// Unrolled version of upper_bound and lower_bound
2022-02-18 15:50:15 +00:00
/// Loosely based on https://academy.realm.io/posts/how-we-beat-cpp-stl-binary-search/
2022-02-18 15:16:29 +00:00
/// In the future it'd interesting to replace it with a B+Tree Layout as described
/// at https://en.algorithmica.org/hpc/data-structures/s-tree/
2022-02-18 15:50:15 +00:00
size_t boundSearch(TKey value)
2022-02-18 13:05:42 +00:00
{
size_t size = array.size();
size_t low = 0;
2022-02-18 15:50:15 +00:00
/// This is a single binary search iteration as a macro to unroll. Takes into account the inequality:
/// isStrict -> Equal values are not requested
/// isDescending -> The vector is sorted in reverse (for greater or greaterOrEquals)
2022-02-18 15:16:29 +00:00
#define BOUND_ITERATION \
{ \
size_t half = size / 2; \
size_t other_half = size - half; \
size_t probe = low + half; \
size_t other_low = low + other_half; \
TKey v = array[probe].asof_value; \
size = half; \
2022-02-18 15:50:15 +00:00
if constexpr (isDescending) \
{ \
if constexpr (isStrict) \
low = value <= v ? other_low : low; \
else \
low = value < v ? other_low : low; \
} \
2022-02-18 15:16:29 +00:00
else \
2022-02-18 15:50:15 +00:00
{ \
if constexpr (isStrict) \
low = value >= v ? other_low : low; \
else \
low = value > v ? other_low : low; \
} \
2022-02-18 15:16:29 +00:00
}
2022-02-18 13:05:42 +00:00
while (size >= 8)
{
2022-02-18 15:16:29 +00:00
BOUND_ITERATION
BOUND_ITERATION
BOUND_ITERATION
2022-02-18 13:05:42 +00:00
}
while (size > 0)
{
2022-02-18 15:16:29 +00:00
BOUND_ITERATION
2022-02-18 13:05:42 +00:00
}
2022-02-18 15:16:29 +00:00
#undef BOUND_ITERATION
2022-02-18 13:05:42 +00:00
return low;
}
2022-02-19 20:01:47 +00:00
std::tuple<decltype(RowRef::block), decltype(RowRef::row_num)> findAsof(const IColumn & asof_column, size_t row_num) override
{
2022-02-04 16:12:01 +00:00
sort();
using ColumnType = ColumnVectorOrDecimal<TKey>;
const auto & column = assert_cast<const ColumnType &>(asof_column);
TKey k = column.getElement(row_num);
2022-02-18 15:50:15 +00:00
size_t pos = boundSearch(k);
if (pos != array.size())
2022-02-19 20:01:47 +00:00
return std::make_tuple(array[pos].block, array[pos].row_num);
2022-02-19 20:01:47 +00:00
return {nullptr, 0};
2019-10-11 17:56:26 +00:00
}
private:
std::atomic<bool> sorted = false;
mutable std::mutex lock;
2022-02-04 16:12:01 +00:00
Base array;
2019-04-25 01:16:26 +00:00
// Double checked locking with SC atomics works in C++
// https://preshing.com/20130930/double-checked-locking-is-fixed-in-cpp11/
// The first thread that calls one of the lookup methods sorts the data
// After calling the first lookup method it is no longer allowed to insert any data
// the array becomes immutable
2022-02-04 16:12:01 +00:00
void sort()
{
2019-04-05 18:05:24 +00:00
if (!sorted.load(std::memory_order_acquire))
{
std::lock_guard<std::mutex> l(lock);
2019-04-05 18:05:24 +00:00
if (!sorted.load(std::memory_order_relaxed))
{
2022-02-18 15:50:15 +00:00
if constexpr (isDescending)
::sort(array.begin(), array.end(), greaterEntryOperator());
else
::sort(array.begin(), array.end());
sorted.store(true, std::memory_order_release);
}
}
}
};
2019-03-30 21:30:21 +00:00
2022-02-18 13:05:42 +00:00
// It only contains a std::unique_ptr which is memmovable.
2022-02-18 10:02:14 +00:00
// Source: https://github.com/ClickHouse/ClickHouse/issues/4906
using AsofRowRefs = std::unique_ptr<SortedLookupVectorBase>;
2022-02-18 15:16:29 +00:00
AsofRowRefs createAsofRowRef(TypeIndex type, ASOF::Inequality inequality);
2019-03-31 10:56:54 +00:00
}