mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge
This commit is contained in:
commit
ab165d3cb7
@ -346,17 +346,18 @@ private:
|
||||
}
|
||||
|
||||
protected:
|
||||
template <typename T1, typename T2>
|
||||
bool match(T1 & events_it, const T2 events_end) const
|
||||
template <typename T>
|
||||
bool match(T & events_it, const T events_end) const
|
||||
{
|
||||
const auto action_begin = std::begin(actions);
|
||||
const auto action_end = std::end(actions);
|
||||
auto action_it = action_begin;
|
||||
|
||||
const auto events_begin = events_it;
|
||||
auto base_it = events_it;
|
||||
|
||||
/// an iterator to action plus an iterator to row in events list plus timestamp at the start of sequence
|
||||
using backtrack_info = std::tuple<decltype(action_it), decltype(events_it), decltype(base_it)>;
|
||||
using backtrack_info = std::tuple<decltype(action_it), T, T>;
|
||||
std::stack<backtrack_info> back_stack;
|
||||
|
||||
/// backtrack if possible
|
||||
@ -473,6 +474,9 @@ protected:
|
||||
++action_it;
|
||||
}
|
||||
|
||||
if (events_it == events_begin)
|
||||
++events_it;
|
||||
|
||||
return action_it == action_end;
|
||||
}
|
||||
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
#include <DB/AggregateFunctions/UniqCombinedBiasData.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -29,7 +30,7 @@ namespace DB
|
||||
|
||||
struct AggregateFunctionUniqUniquesHashSetData
|
||||
{
|
||||
typedef UniquesHashSet<DefaultHash<UInt64>> Set;
|
||||
typedef UniquesHashSet<DefaultHash<UInt64> > Set;
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniq"; }
|
||||
@ -94,83 +95,79 @@ struct AggregateFunctionUniqExactData<String>
|
||||
static String getName() { return "uniqExact"; }
|
||||
};
|
||||
|
||||
template <typename T, HyperLogLogMode mode>
|
||||
struct BaseUniqCombinedData
|
||||
{
|
||||
using Key = UInt64;
|
||||
using Set = CombinedCardinalityEstimator<
|
||||
Key,
|
||||
HashSet<Key, DefaultHash<Key>, HashTableGrower<> >,
|
||||
16,
|
||||
14,
|
||||
17,
|
||||
DefaultHash<Key>,
|
||||
UInt64,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
mode
|
||||
>;
|
||||
|
||||
Set set;
|
||||
};
|
||||
|
||||
template <HyperLogLogMode mode>
|
||||
struct BaseUniqCombinedData<String, mode>
|
||||
{
|
||||
using Key = UInt64;
|
||||
using Set = CombinedCardinalityEstimator<
|
||||
Key,
|
||||
HashSet<Key, TrivialHash, HashTableGrower<> >,
|
||||
16,
|
||||
14,
|
||||
17,
|
||||
TrivialHash,
|
||||
UInt64,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
mode
|
||||
>;
|
||||
|
||||
Set set;
|
||||
};
|
||||
|
||||
/// Агрегатные функции uniqCombinedRaw, uniqCombinedLinearCounting, и uniqCombinedBiasCorrected
|
||||
/// предназначены для разработки новых версий функции uniqCombined.
|
||||
/// Пользователи должны использовать только uniqCombined.
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedRawData
|
||||
: public BaseUniqCombinedData<T, HyperLogLogMode::Raw>
|
||||
{
|
||||
static String getName() { return "uniqCombinedRaw"; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedLinearCountingData
|
||||
: public BaseUniqCombinedData<T, HyperLogLogMode::LinearCounting>
|
||||
{
|
||||
static String getName() { return "uniqCombinedLinearCounting"; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedBiasCorrectedData
|
||||
: public BaseUniqCombinedData<T, HyperLogLogMode::BiasCorrected>
|
||||
{
|
||||
static String getName() { return "uniqCombinedBiasCorrected"; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedData
|
||||
: public BaseUniqCombinedData<T, HyperLogLogMode::FullFeatured>
|
||||
{
|
||||
using Key = UInt32;
|
||||
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, HashTableGrower<> >, 16, 14, 17, TrivialHash>;
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniqCombined"; }
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedData<String>
|
||||
{
|
||||
using Key = UInt64;
|
||||
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, HashTableGrower<> >, 16, 14, 17, TrivialHash>;
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniqCombined"; }
|
||||
};
|
||||
|
||||
namespace detail
|
||||
{
|
||||
|
||||
/** Хэш-функция для uniqCombined.
|
||||
*/
|
||||
template<typename T, typename Enable = void>
|
||||
struct CombinedCardinalityTraits
|
||||
{
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
return key;
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, Int64>::value>::type>
|
||||
{
|
||||
using U = typename std::make_unsigned<T>::type;
|
||||
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
return intHash32<0>(static_cast<U>(key));
|
||||
};
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, UInt64>::value>::type>
|
||||
{
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
return intHash32<0>(key);
|
||||
};
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, Float64>::value>::type>
|
||||
{
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
UInt64 res = 0;
|
||||
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&key), sizeof(key));
|
||||
return intHash32<0>(res);
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, Float32>::value>::type>
|
||||
{
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
UInt32 res = 0;
|
||||
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&key), sizeof(key));
|
||||
return res;
|
||||
}
|
||||
};
|
||||
|
||||
/** Хэш-функция для uniq.
|
||||
*/
|
||||
template <typename T> struct AggregateFunctionUniqTraits
|
||||
@ -199,41 +196,51 @@ template <> struct AggregateFunctionUniqTraits<Float64>
|
||||
};
|
||||
|
||||
/** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq.
|
||||
* Используется для частичной специализации для добавления строк.
|
||||
*/
|
||||
template<typename T, typename Data>
|
||||
struct OneAdder
|
||||
{
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num)
|
||||
{
|
||||
data.set.insert(AggregateFunctionUniqTraits<T>::hash(static_cast<const ColumnVector<T> &>(column).getData()[row_num]));
|
||||
}
|
||||
};
|
||||
* Используется для частичной специализации для добавления строк.
|
||||
*/
|
||||
template <typename T, typename Data, typename Enable = void>
|
||||
struct OneAdder;
|
||||
|
||||
template<typename Data>
|
||||
struct OneAdder<String, Data>
|
||||
template <typename T, typename Data>
|
||||
struct OneAdder<T, Data, typename std::enable_if<
|
||||
std::is_same<Data, AggregateFunctionUniqUniquesHashSetData>::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqHLL12Data<T> >::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqCombinedRawData<T> >::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqCombinedLinearCountingData<T> >::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqCombinedBiasCorrectedData<T> >::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqCombinedData<T> >::value>::type>
|
||||
{
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num)
|
||||
template <typename T2 = T>
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num,
|
||||
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
|
||||
{
|
||||
const auto & value = static_cast<const ColumnVector<T2> &>(column).getData()[row_num];
|
||||
data.set.insert(AggregateFunctionUniqTraits<T2>::hash(value));
|
||||
}
|
||||
|
||||
template <typename T2 = T>
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num,
|
||||
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
|
||||
{
|
||||
/// Имейте ввиду, что вычисление приближённое.
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
data.set.insert(CityHash64(value.data, value.size));
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct OneAdder<T, AggregateFunctionUniqExactData<T> >
|
||||
template <typename T, typename Data>
|
||||
struct OneAdder<T, Data, typename std::enable_if<
|
||||
std::is_same<Data, AggregateFunctionUniqExactData<T> >::value>::type>
|
||||
{
|
||||
static void addOne(AggregateFunctionUniqExactData<T> & data, const IColumn & column, size_t row_num)
|
||||
template <typename T2 = T>
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num,
|
||||
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
|
||||
{
|
||||
data.set.insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
|
||||
data.set.insert(static_cast<const ColumnVector<T2> &>(column).getData()[row_num]);
|
||||
}
|
||||
};
|
||||
|
||||
template<>
|
||||
struct OneAdder<String, AggregateFunctionUniqExactData<String> >
|
||||
{
|
||||
static void addOne(AggregateFunctionUniqExactData<String> & data, const IColumn & column, size_t row_num)
|
||||
template <typename T2 = T>
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num,
|
||||
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
|
||||
{
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
|
||||
@ -246,26 +253,6 @@ struct OneAdder<String, AggregateFunctionUniqExactData<String> >
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct OneAdder<T, AggregateFunctionUniqCombinedData<T> >
|
||||
{
|
||||
static void addOne(AggregateFunctionUniqCombinedData<T> & data, const IColumn & column, size_t row_num)
|
||||
{
|
||||
const auto & value = static_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
||||
data.set.insert(CombinedCardinalityTraits<T>::hash(value));
|
||||
}
|
||||
};
|
||||
|
||||
template<>
|
||||
struct OneAdder<String, AggregateFunctionUniqCombinedData<String> >
|
||||
{
|
||||
static void addOne(AggregateFunctionUniqCombinedData<String> & data, const IColumn & column, size_t row_num)
|
||||
{
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
data.set.insert(CityHash64(value.data, value.size));
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -32,8 +32,8 @@ enum class ReservoirSamplerDeterministicOnEmpty
|
||||
RETURN_NAN_OR_ZERO,
|
||||
};
|
||||
|
||||
template<typename T,
|
||||
ReservoirSamplerDeterministicOnEmpty OnEmpty = ReservoirSamplerDeterministicOnEmpty::THROW>
|
||||
template <typename T,
|
||||
ReservoirSamplerDeterministicOnEmpty OnEmpty = ReservoirSamplerDeterministicOnEmpty::THROW>
|
||||
class ReservoirSamplerDeterministic
|
||||
{
|
||||
bool good(const UInt32 hash)
|
||||
@ -65,40 +65,6 @@ public:
|
||||
++total_values;
|
||||
}
|
||||
|
||||
void insertImpl(const T & v, const UInt32 hash)
|
||||
{
|
||||
while (samples.size() + 1 >= sample_count)
|
||||
{
|
||||
if (++skip_degree > detail::MAX_SKIP_DEGREE)
|
||||
throw DB::Exception{"skip_degree exceeds maximum value", DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED};
|
||||
thinOut();
|
||||
}
|
||||
|
||||
samples.emplace_back(v, hash);
|
||||
}
|
||||
|
||||
void thinOut()
|
||||
{
|
||||
auto size = samples.size();
|
||||
for (size_t i = 0; i < size;)
|
||||
{
|
||||
if (!good(samples[i].second))
|
||||
{
|
||||
/// swap current element with the last one
|
||||
std::swap(samples[size - 1], samples[i]);
|
||||
--size;
|
||||
}
|
||||
else
|
||||
++i;
|
||||
}
|
||||
|
||||
if (size != samples.size())
|
||||
{
|
||||
samples.resize(size);
|
||||
sorted = false;
|
||||
}
|
||||
}
|
||||
|
||||
size_t size() const
|
||||
{
|
||||
return total_values;
|
||||
@ -119,6 +85,7 @@ public:
|
||||
|
||||
/** Если T не числовой тип, использование этого метода вызывает ошибку компиляции,
|
||||
* но использование класса ошибки не вызывает. SFINAE.
|
||||
* Не SFINAE. Функции члены шаблонов типов просто не проверяются, пока не используются.
|
||||
*/
|
||||
double quantileInterpolated(double level)
|
||||
{
|
||||
@ -153,9 +120,9 @@ public:
|
||||
thinOut();
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < b.samples.size(); ++i)
|
||||
if (good(b.samples[i].second))
|
||||
insertImpl(b.samples[i].first, b.samples[i].second);
|
||||
for (const auto & sample : b.samples)
|
||||
if (good(sample.second))
|
||||
insertImpl(sample.first, sample.second);
|
||||
|
||||
total_values += b.total_values;
|
||||
}
|
||||
@ -167,7 +134,7 @@ public:
|
||||
samples.resize(std::min(total_values, sample_count));
|
||||
|
||||
for (size_t i = 0; i < samples.size(); ++i)
|
||||
DB::readBinary(samples[i].first, buf);
|
||||
DB::readPODBinary(samples[i], buf);
|
||||
|
||||
sorted = false;
|
||||
}
|
||||
@ -178,7 +145,7 @@ public:
|
||||
DB::writeIntBinary<size_t>(total_values, buf);
|
||||
|
||||
for (size_t i = 0; i < std::min(sample_count, total_values); ++i)
|
||||
DB::writeBinary(samples[i].first, buf);
|
||||
DB::writePODBinary(samples[i], buf);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -191,6 +158,41 @@ private:
|
||||
std::vector<std::pair<T, UInt32>> samples;
|
||||
UInt8 skip_degree{};
|
||||
|
||||
void insertImpl(const T & v, const UInt32 hash)
|
||||
{
|
||||
/// @todo why + 1? I don't quite recall
|
||||
while (samples.size() + 1 >= sample_count)
|
||||
{
|
||||
if (++skip_degree > detail::MAX_SKIP_DEGREE)
|
||||
throw DB::Exception{"skip_degree exceeds maximum value", DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED};
|
||||
thinOut();
|
||||
}
|
||||
|
||||
samples.emplace_back(v, hash);
|
||||
}
|
||||
|
||||
void thinOut()
|
||||
{
|
||||
auto size = samples.size();
|
||||
for (size_t i = 0; i < size;)
|
||||
{
|
||||
if (!good(samples[i].second))
|
||||
{
|
||||
/// swap current element with the last one
|
||||
std::swap(samples[size - 1], samples[i]);
|
||||
--size;
|
||||
}
|
||||
else
|
||||
++i;
|
||||
}
|
||||
|
||||
if (size != samples.size())
|
||||
{
|
||||
samples.resize(size);
|
||||
sorted = false;
|
||||
}
|
||||
}
|
||||
|
||||
void sortIfNeeded()
|
||||
{
|
||||
if (sorted)
|
||||
|
39
dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h
Normal file
39
dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
|
||||
#include <array>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Данные для HyperLogLogBiasEstimator в функции uniqCombined.
|
||||
* Схема разработки следующая:
|
||||
* 1. Собрать ClickHouse.
|
||||
* 2. Запустить скрипт src/dbms/scripts/gen-bias-data.py, который возвращает один массив для getRawEstimates()
|
||||
* и другой массив для getBiases().
|
||||
* 3. Обновить массивы raw_estimates и biases. Также обновить размер массивов в InterpolatedData.
|
||||
* 4. Собрать ClickHouse.
|
||||
* 5. Запустить скрипт src/dbms/scripts/linear-counting-threshold.py, который создаёт 3 файла:
|
||||
* - raw_graph.txt (1-й столбец: настоящее количество уникальных значений;
|
||||
* 2-й столбец: относительная погрешность в случае HyperLogLog без применения каких-либо поправок)
|
||||
* - linear_counting_graph.txt (1-й столбец: настоящее количество уникальных значений;
|
||||
* 2-й столбец: относительная погрешность в случае HyperLogLog с применением LinearCounting)
|
||||
* - bias_corrected_graph.txt (1-й столбец: настоящее количество уникальных значений;
|
||||
* 2-й столбец: относительная погрешность в случае HyperLogLog с применением поправок из алгортима HyperLogLog++)
|
||||
* 6. Сгенерить график с gnuplot на основе этих данных.
|
||||
* 7. Определить минимальное количество уникальных значений, при котором лучше исправить погрешность
|
||||
* с помощью её оценки (т.е. по алгоритму HyperLogLog++), чем применить алгоритм LinearCounting.
|
||||
* 7. Соответственно обновить константу в функции getThreshold()
|
||||
* 8. Собрать ClickHouse.
|
||||
*/
|
||||
struct UniqCombinedBiasData
|
||||
{
|
||||
using InterpolatedData = std::array<double, 178>;
|
||||
|
||||
static double getThreshold();
|
||||
/// Оценки количества уникальных значений по алгоритму HyperLogLog без применения каких-либо поправок.
|
||||
static const InterpolatedData & getRawEstimates();
|
||||
/// Соответствующие оценки погрешности.
|
||||
static const InterpolatedData & getBiases();
|
||||
};
|
||||
|
||||
}
|
@ -70,6 +70,9 @@ private:
|
||||
/// Внутренняя версия функции receivePacket без блокировки.
|
||||
Connection::Packet receivePacketUnlocked();
|
||||
|
||||
/// Внутренняя версия функции dumpAddresses без блокировки.
|
||||
std::string dumpAddressesUnlocked() const;
|
||||
|
||||
/// Получить реплику, на которой можно прочитать данные.
|
||||
ReplicaMap::iterator getReplicaForReading();
|
||||
|
||||
|
@ -100,9 +100,9 @@ public:
|
||||
arenas.push_back(arena_);
|
||||
}
|
||||
|
||||
ColumnPtr convertToValues()
|
||||
ColumnPtr convertToValues() const
|
||||
{
|
||||
IAggregateFunction * function = holder->func;
|
||||
const IAggregateFunction * function = holder->func;
|
||||
ColumnPtr res = function->getReturnType()->createColumn();
|
||||
IColumn & column = *res;
|
||||
res->reserve(getData().size());
|
||||
|
@ -36,21 +36,16 @@ public:
|
||||
|
||||
if (size >= MMAP_THRESHOLD)
|
||||
{
|
||||
if (alignment <= MMAP_MIN_ALIGNMENT)
|
||||
{
|
||||
buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
|
||||
if (MAP_FAILED == buf)
|
||||
DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
if (alignment > MMAP_MIN_ALIGNMENT)
|
||||
throw DB::Exception("Too large alignment: more than page size.", DB::ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
/// См. комментарий в HashTableAllocator.h
|
||||
if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE))
|
||||
DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// если выравнивание больше, чем размер страницы используем posix_memalign
|
||||
buf = allocateUsingPosixMemalign(size, alignment);
|
||||
}
|
||||
buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
|
||||
if (MAP_FAILED == buf)
|
||||
DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
|
||||
/// См. комментарий в HashTableAllocator.h
|
||||
if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE))
|
||||
DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -63,7 +58,11 @@ public:
|
||||
}
|
||||
else
|
||||
{
|
||||
buf = allocateUsingPosixMemalign(size, alignment);
|
||||
buf = nullptr;
|
||||
int res = posix_memalign(&buf, alignment, size);
|
||||
|
||||
if (0 != res)
|
||||
DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
|
||||
}
|
||||
}
|
||||
|
||||
@ -122,17 +121,4 @@ public:
|
||||
|
||||
return buf;
|
||||
}
|
||||
|
||||
private:
|
||||
void * allocateUsingPosixMemalign(size_t size, size_t alignment)
|
||||
{
|
||||
void * buf = nullptr;
|
||||
|
||||
int res = posix_memalign(&buf, alignment, size);
|
||||
|
||||
if (0 != res)
|
||||
DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
|
||||
|
||||
return buf;
|
||||
}
|
||||
};
|
||||
|
@ -34,17 +34,32 @@ template
|
||||
UInt8 medium_set_power2_max,
|
||||
UInt8 K,
|
||||
typename Hash = IntHash32<Key>,
|
||||
typename HashValueType = UInt32,
|
||||
typename BiasEstimator = TrivialBiasEstimator,
|
||||
HyperLogLogMode mode = HyperLogLogMode::FullFeatured,
|
||||
typename DenominatorType = double
|
||||
>
|
||||
class CombinedCardinalityEstimator
|
||||
{
|
||||
public:
|
||||
using Self = CombinedCardinalityEstimator<Key, HashContainer, small_set_size_max, medium_set_power2_max, K, Hash, DenominatorType>;
|
||||
using Self = CombinedCardinalityEstimator
|
||||
<
|
||||
Key,
|
||||
HashContainer,
|
||||
small_set_size_max,
|
||||
medium_set_power2_max,
|
||||
K,
|
||||
Hash,
|
||||
HashValueType,
|
||||
BiasEstimator,
|
||||
mode,
|
||||
DenominatorType
|
||||
>;
|
||||
|
||||
private:
|
||||
using Small = SmallSet<Key, small_set_size_max>;
|
||||
using Medium = HashContainer;
|
||||
using Large = HyperLogLogCounter<K, Hash, DenominatorType>;
|
||||
using Large = HyperLogLogCounter<K, Hash, HashValueType, DenominatorType, BiasEstimator, mode>;
|
||||
|
||||
public:
|
||||
CombinedCardinalityEstimator()
|
||||
|
@ -167,7 +167,7 @@ public:
|
||||
|
||||
ExternalTablesHandler(Context & context_, Poco::Net::NameValueCollection params_) : context(context_), params(params_) { }
|
||||
|
||||
void handlePart(const Poco::Net::MessageHeader& header, std::istream& stream)
|
||||
void handlePart(const Poco::Net::MessageHeader & header, std::istream & stream)
|
||||
{
|
||||
/// Буфер инициализируется здесь, а не в виртуальной функции initReadBuffer
|
||||
read_buffer.reset(new ReadBufferFromIStream(stream));
|
||||
@ -194,7 +194,7 @@ public:
|
||||
NamesAndTypesListPtr columns = new NamesAndTypesList(sample_block.getColumnsList());
|
||||
StoragePtr storage = StorageMemory::create(data.second, columns);
|
||||
context.addExternalTable(data.second, storage);
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr());
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr(), context.getSettingsRef());
|
||||
|
||||
/// Записываем данные
|
||||
data.first->readPrefix();
|
||||
|
@ -443,16 +443,18 @@ public:
|
||||
|
||||
bool next()
|
||||
{
|
||||
if (!is_initialized)
|
||||
{
|
||||
Cell::State::read(in);
|
||||
DB::readVarUInt(size, in);
|
||||
is_initialized = true;
|
||||
}
|
||||
|
||||
if (read_count == size)
|
||||
{
|
||||
is_eof = true;
|
||||
return false;
|
||||
}
|
||||
else if (read_count == 0)
|
||||
{
|
||||
Cell::State::read(in);
|
||||
DB::readVarUInt(size, in);
|
||||
}
|
||||
|
||||
cell.read(in);
|
||||
++read_count;
|
||||
@ -462,18 +464,19 @@ public:
|
||||
|
||||
inline const value_type & get() const
|
||||
{
|
||||
if ((read_count == 0) || is_eof)
|
||||
if (!is_initialized || is_eof)
|
||||
throw DB::Exception("No available data", DB::ErrorCodes::NO_AVAILABLE_DATA);
|
||||
|
||||
return cell.getValue();
|
||||
}
|
||||
|
||||
private:
|
||||
DB::ReadBuffer in;
|
||||
DB::ReadBuffer & in;
|
||||
Cell cell;
|
||||
size_t read_count = 0;
|
||||
size_t size;
|
||||
bool is_eof = false;
|
||||
bool is_initialized = false;
|
||||
};
|
||||
|
||||
class iterator
|
||||
|
@ -80,18 +80,21 @@ public:
|
||||
|
||||
bool next()
|
||||
{
|
||||
if (read_count == size)
|
||||
{
|
||||
is_eof = true;
|
||||
return false;
|
||||
}
|
||||
else if (read_count == 0)
|
||||
if (!is_initialized)
|
||||
{
|
||||
Cell::State::read(in);
|
||||
DB::readVarUInt(size, in);
|
||||
|
||||
if (size > capacity)
|
||||
throw DB::Exception("Illegal size");
|
||||
|
||||
is_initialized = true;
|
||||
}
|
||||
|
||||
if (read_count == size)
|
||||
{
|
||||
is_eof = true;
|
||||
return false;
|
||||
}
|
||||
|
||||
cell.read(in);
|
||||
@ -102,18 +105,19 @@ public:
|
||||
|
||||
inline const value_type & get() const
|
||||
{
|
||||
if ((read_count == 0) || is_eof)
|
||||
if (!is_initialized || is_eof)
|
||||
throw DB::Exception("No available data", DB::ErrorCodes::NO_AVAILABLE_DATA);
|
||||
|
||||
return cell.getValue();
|
||||
}
|
||||
|
||||
private:
|
||||
DB::ReadBuffer in;
|
||||
DB::ReadBuffer & in;
|
||||
Cell cell;
|
||||
size_t read_count = 0;
|
||||
size_t size;
|
||||
bool is_eof = false;
|
||||
bool is_initialized = false;
|
||||
};
|
||||
|
||||
class iterator
|
||||
|
@ -53,7 +53,7 @@ public:
|
||||
size_t hash(const Key & x) const { return Hash::operator()(x); }
|
||||
|
||||
/// NOTE Плохо для хэш-таблиц больше чем на 2^32 ячеек.
|
||||
size_t getBucketFromHash(size_t hash_value) const { return (hash_value >> (32 - BITS_FOR_BUCKET)) & MAX_BUCKET; }
|
||||
static size_t getBucketFromHash(size_t hash_value) { return (hash_value >> (32 - BITS_FOR_BUCKET)) & MAX_BUCKET; }
|
||||
|
||||
protected:
|
||||
typename Impl::iterator beginOfNextNonEmptyBucket(size_t & bucket)
|
||||
|
@ -21,7 +21,7 @@ class HyperLogLogWithSmallSetOptimization
|
||||
{
|
||||
private:
|
||||
using Small = SmallSet<Key, small_set_size>;
|
||||
using Large = HyperLogLogCounter<K, Hash, DenominatorType>;
|
||||
using Large = HyperLogLogCounter<K, Hash, UInt32, DenominatorType>;
|
||||
|
||||
Small small;
|
||||
Large * large = nullptr;
|
||||
|
@ -45,7 +45,7 @@
|
||||
*/
|
||||
#define DEFAULT_MERGE_BLOCK_SIZE 8192
|
||||
|
||||
#define DEFAULT_MAX_QUERY_SIZE 65536
|
||||
#define DEFAULT_MAX_QUERY_SIZE 262144
|
||||
#define SHOW_CHARS_ON_SYNTAX_ERROR 160L
|
||||
#define DEFAULT_MAX_DISTRIBUTED_CONNECTIONS 1024
|
||||
#define DEFAULT_INTERACTIVE_DELAY 100000
|
||||
|
@ -20,7 +20,6 @@ namespace ErrorCodes
|
||||
POSITION_OUT_OF_BOUND = 11,
|
||||
PARAMETER_OUT_OF_BOUND = 12,
|
||||
SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH = 13,
|
||||
EMPTY_TUPLE = 14,
|
||||
DUPLICATE_COLUMN = 15,
|
||||
NO_SUCH_COLUMN_IN_TABLE = 16,
|
||||
DELIMITER_IN_STRING_LITERAL_DOESNT_MATCH = 17,
|
||||
@ -286,6 +285,12 @@ namespace ErrorCodes
|
||||
NO_AVAILABLE_DATA = 280,
|
||||
DICTIONARY_IS_EMPTY = 281,
|
||||
INCORRECT_INDEX = 282,
|
||||
UNKNOWN_DISTRIBUTED_PRODUCT_MODE = 283,
|
||||
UNKNOWN_GLOBAL_SUBQUERIES_METHOD = 284,
|
||||
TOO_LESS_LIVE_REPLICAS = 285,
|
||||
UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE = 286,
|
||||
UNKNOWN_FORMAT_VERSION = 287,
|
||||
DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED = 288,
|
||||
|
||||
KEEPER_EXCEPTION = 999,
|
||||
POCO_EXCEPTION = 1000,
|
||||
|
@ -55,6 +55,9 @@ protected:
|
||||
Block res = children.back()->read();
|
||||
if (!res)
|
||||
return res;
|
||||
/** @todo if somehow block does not contain values for implicitly-defaulted columns that are prerequisites
|
||||
* for explicitly-defaulted ones, exception will be thrown during evaluating such columns
|
||||
* (implicitly-defaulted columns are evaluated on the line after following one. */
|
||||
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
|
||||
res.addDefaults(*required_columns);
|
||||
return res;
|
||||
|
@ -37,10 +37,14 @@ public:
|
||||
Block res = block;
|
||||
|
||||
/// Вычисляет явно указанные (в column_defaults) значения по-умолчанию.
|
||||
/** @todo if somehow block does not contain values for implicitly-defaulted columns that are prerequisites
|
||||
* for explicitly-defaulted ones, exception will be thrown during evaluating such columns
|
||||
* (implicitly-defaulted columns are evaluated on the line after following one. */
|
||||
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
|
||||
|
||||
/// Добавляет не указанные значения по-умолчанию.
|
||||
if (!only_explicit_column_defaults)
|
||||
/// @todo this line may be moved before `evaluateMissingDefaults` with passing {required_columns - explicitly-defaulted columns}
|
||||
res.addDefaults(*required_columns);
|
||||
|
||||
output->write(res);
|
||||
|
@ -38,6 +38,15 @@ protected:
|
||||
return Block();
|
||||
|
||||
children.push_back(input);
|
||||
|
||||
if (IProfilingBlockInputStream * p_input = dynamic_cast<IProfilingBlockInputStream *>(input.get()))
|
||||
{
|
||||
/// Они могли быть установлены раньше, но не были протащены в input.
|
||||
if (progress_callback)
|
||||
p_input->setProgressCallback(progress_callback);
|
||||
if (process_list_elem)
|
||||
p_input->setProcessListElement(process_list_elem);
|
||||
}
|
||||
}
|
||||
|
||||
return input->read();
|
||||
|
@ -16,7 +16,12 @@ using Poco::SharedPtr;
|
||||
class LimitBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_ = 0);
|
||||
/** Если always_read_till_end = false (по-умолчанию), то после чтения достаточного количества данных,
|
||||
* возвращает пустой блок, и это приводит к отмене выполнения запроса.
|
||||
* Если always_read_till_end = true - читает все данные до конца, но игнорирует их. Это нужно в редких случаях:
|
||||
* когда иначе, из-за отмены запроса, мы бы не получили данные для GROUP BY WITH TOTALS с удалённого сервера.
|
||||
*/
|
||||
LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_ = false);
|
||||
|
||||
String getName() const override { return "Limit"; }
|
||||
|
||||
@ -33,7 +38,8 @@ protected:
|
||||
private:
|
||||
size_t limit;
|
||||
size_t offset;
|
||||
size_t pos;
|
||||
size_t pos = 0;
|
||||
bool always_read_till_end;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -12,99 +12,54 @@ namespace DB
|
||||
* Это экономит оперативку в случае использования двухуровневой агрегации, где в каждом потоке будет до 256 блоков с частями результата.
|
||||
*
|
||||
* Агрегатные функции в блоках не должны быть финализированы, чтобы их состояния можно было объединить.
|
||||
*
|
||||
* Замечания:
|
||||
*
|
||||
* На хорошей сети (10Gbit) может работать заметно медленнее, так как чтения блоков с разных
|
||||
* удалённых серверов делаются последовательно, при этом, чтение упирается в CPU.
|
||||
* Это несложно исправить.
|
||||
*
|
||||
* Также, чтения и вычисления (слияние состояний) делаются по очереди.
|
||||
* Есть возможность делать чтения асинхронно - при этом будет расходоваться в два раза больше памяти, но всё-равно немного.
|
||||
* Это можно сделать с помощью UnionBlockInputStream.
|
||||
*
|
||||
* Можно держать в памяти не по одному блоку из каждого источника, а по несколько, и распараллелить мердж.
|
||||
* При этом будет расходоваться кратно больше оперативки.
|
||||
*/
|
||||
class MergingAggregatedMemoryEfficientBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
MergingAggregatedMemoryEfficientBlockInputStream(BlockInputStreams inputs_, const Names & keys_names_,
|
||||
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_)
|
||||
: aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0),
|
||||
final(final_)
|
||||
{
|
||||
children = inputs_;
|
||||
current_blocks.resize(children.size());
|
||||
overflow_blocks.resize(children.size());
|
||||
is_exhausted.resize(children.size());
|
||||
}
|
||||
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_);
|
||||
|
||||
String getName() const override { return "MergingAggregatedMemorySavvy"; }
|
||||
String getName() const override { return "MergingAggregatedMemoryEfficient"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
std::stringstream res;
|
||||
res << "MergingAggregatedMemorySavvy(" << aggregator.getID();
|
||||
for (size_t i = 0, size = children.size(); i < size; ++i)
|
||||
res << ", " << children.back()->getID();
|
||||
res << ")";
|
||||
return res.str();
|
||||
}
|
||||
String getID() const override;
|
||||
|
||||
protected:
|
||||
Block readImpl() override
|
||||
{
|
||||
/// Если child - RemoteBlockInputStream, то отправляет запрос на все удалённые серверы, инициируя вычисления.
|
||||
if (current_bucket_num == -1)
|
||||
for (auto & child : children)
|
||||
child->readPrefix();
|
||||
|
||||
/// Всё прочитали.
|
||||
if (current_bucket_num > 255)
|
||||
return {};
|
||||
|
||||
/// Читаем следующие блоки для current_bucket_num
|
||||
for (size_t i = 0, size = children.size(); i < size; ++i)
|
||||
{
|
||||
while (!is_exhausted[i] && (!current_blocks[i] || current_blocks[i].info.bucket_num < current_bucket_num))
|
||||
{
|
||||
current_blocks[i] = children[i]->read();
|
||||
|
||||
if (!current_blocks[i])
|
||||
{
|
||||
is_exhausted[i] = true;
|
||||
}
|
||||
else if (current_blocks[i].info.is_overflows)
|
||||
{
|
||||
overflow_blocks[i].swap(current_blocks[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Может быть, нет блоков для current_bucket_num, а все блоки имеют больший bucket_num.
|
||||
Int32 min_bucket_num = 256;
|
||||
for (size_t i = 0, size = children.size(); i < size; ++i)
|
||||
if (!is_exhausted[i] && current_blocks[i].info.bucket_num < min_bucket_num)
|
||||
min_bucket_num = current_blocks[i].info.bucket_num;
|
||||
|
||||
current_bucket_num = min_bucket_num;
|
||||
|
||||
/// Все потоки исчерпаны.
|
||||
if (current_bucket_num > 255)
|
||||
return {}; /// TODO overflow_blocks.
|
||||
|
||||
/// TODO Если есть single_level и two_level блоки.
|
||||
|
||||
/// Объединяем все блоки с current_bucket_num.
|
||||
|
||||
BlocksList blocks_to_merge;
|
||||
for (size_t i = 0, size = children.size(); i < size; ++i)
|
||||
if (current_blocks[i].info.bucket_num == current_bucket_num)
|
||||
blocks_to_merge.emplace_back(std::move(current_blocks[i]));
|
||||
|
||||
Block res = aggregator.mergeBlocks(blocks_to_merge, final);
|
||||
|
||||
++current_bucket_num;
|
||||
return res;
|
||||
}
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
Aggregator aggregator;
|
||||
bool final;
|
||||
|
||||
Int32 current_bucket_num = -1;
|
||||
std::vector<Block> current_blocks;
|
||||
std::vector<UInt8> is_exhausted;
|
||||
bool started = false;
|
||||
bool has_two_level = false;
|
||||
bool has_overflows = false;
|
||||
int current_bucket_num = -1;
|
||||
|
||||
std::vector<Block> overflow_blocks;
|
||||
struct Input
|
||||
{
|
||||
BlockInputStreamPtr stream;
|
||||
Block block;
|
||||
Block overflow_block;
|
||||
std::vector<Block> splitted_blocks;
|
||||
bool is_exhausted = false;
|
||||
|
||||
Input(BlockInputStreamPtr & stream_) : stream(stream_) {}
|
||||
};
|
||||
|
||||
std::vector<Input> inputs;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -22,6 +22,11 @@ public:
|
||||
: context(context_), query_ptr(query_ptr_)
|
||||
{
|
||||
storage = context.getTable(database, table);
|
||||
|
||||
/** TODO Это очень важная строчка. При любой вставке в таблицу один из stream-ов должен владеть lock-ом.
|
||||
* Хотя сейчас любая вставка в таблицу делается через PushingToViewsBlockOutputStream,
|
||||
* но ясно, что здесь - не лучшее место для этой функциональности.
|
||||
*/
|
||||
addTableLock(storage->lockStructure(true));
|
||||
|
||||
Dependencies dependencies = context.getDependencies(database, table);
|
||||
@ -32,7 +37,7 @@ public:
|
||||
}
|
||||
|
||||
if (storage->getName() != "View")
|
||||
output = storage->write(query_ptr);
|
||||
output = storage->write(query_ptr, context.getSettingsRef());
|
||||
}
|
||||
|
||||
void write(const Block & block) override
|
||||
|
@ -120,7 +120,7 @@ public:
|
||||
}
|
||||
|
||||
protected:
|
||||
/// Отправить на удаленные реплики все временные таблицы
|
||||
/// Отправить на удаленные серверы все временные таблицы.
|
||||
void sendExternalTables()
|
||||
{
|
||||
size_t count = parallel_replicas->size();
|
||||
@ -151,6 +151,7 @@ protected:
|
||||
parallel_replicas->sendExternalTablesData(external_tables_data);
|
||||
}
|
||||
|
||||
|
||||
Block readImpl() override
|
||||
{
|
||||
if (!sent_query)
|
||||
|
@ -7,6 +7,8 @@
|
||||
#include <DB/Common/isLocalAddress.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include "writeParenthesisedString.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -93,14 +95,35 @@ private:
|
||||
WriteBufferFromString out{query};
|
||||
writeString("SELECT ", out);
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id_name, out);
|
||||
if (!dict_struct.id.expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.id.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
if (!dict_struct.range_min.empty() && !dict_struct.range_max.empty())
|
||||
writeProbablyBackQuotedString(dict_struct.id.name, out);
|
||||
|
||||
if (dict_struct.range_min && dict_struct.range_max)
|
||||
{
|
||||
writeString(", ", out);
|
||||
writeProbablyBackQuotedString(dict_struct.range_min, out);
|
||||
|
||||
if (!dict_struct.range_min->expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.range_min->expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.range_min->name, out);
|
||||
|
||||
writeString(", ", out);
|
||||
writeProbablyBackQuotedString(dict_struct.range_max, out);
|
||||
|
||||
if (!dict_struct.range_max->expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.range_max->expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.range_max->name, out);
|
||||
}
|
||||
|
||||
for (const auto & attr : dict_struct.attributes)
|
||||
@ -109,7 +132,7 @@ private:
|
||||
|
||||
if (!attr.expression.empty())
|
||||
{
|
||||
writeString(attr.expression, out);
|
||||
writeParenthesisedString(attr.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
@ -144,7 +167,13 @@ private:
|
||||
WriteBufferFromString out{query};
|
||||
writeString("SELECT ", out);
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id_name, out);
|
||||
if (!dict_struct.id.expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.id.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id.name, out);
|
||||
|
||||
for (const auto & attr : dict_struct.attributes)
|
||||
{
|
||||
@ -152,7 +181,7 @@ private:
|
||||
|
||||
if (!attr.expression.empty())
|
||||
{
|
||||
writeString(attr.expression, out);
|
||||
writeParenthesisedString(attr.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
@ -175,7 +204,7 @@ private:
|
||||
writeString(" AND ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id_name, out);
|
||||
writeProbablyBackQuotedString(dict_struct.id.name, out);
|
||||
writeString(" IN (", out);
|
||||
|
||||
auto first = true;
|
||||
|
@ -21,10 +21,18 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
|
||||
ColumnWithTypeAndName{
|
||||
new ColumnUInt64,
|
||||
new DataTypeUInt64,
|
||||
dict_struct.id_name
|
||||
dict_struct.id.name
|
||||
}
|
||||
};
|
||||
|
||||
if (dict_struct.range_min)
|
||||
for (const auto & attribute : { dict_struct.range_min, dict_struct.range_max })
|
||||
block.insert(ColumnWithTypeAndName{
|
||||
new ColumnUInt16,
|
||||
new DataTypeDate,
|
||||
attribute->name
|
||||
});
|
||||
|
||||
for (const auto & attribute : dict_struct.attributes)
|
||||
block.insert(ColumnWithTypeAndName{
|
||||
attribute.type->createColumn(), attribute.type, attribute.name
|
||||
@ -57,6 +65,12 @@ public:
|
||||
|
||||
if ("file" == source_type)
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{
|
||||
"Dictionary source of type `file` does not support attribute expressions",
|
||||
ErrorCodes::LOGICAL_ERROR
|
||||
};
|
||||
|
||||
const auto filename = config.getString(config_prefix + ".file.path");
|
||||
const auto format = config.getString(config_prefix + ".file.format");
|
||||
return std::make_unique<FileDictionarySource>(filename, format, sample_block, context);
|
||||
|
@ -7,6 +7,8 @@
|
||||
#include <vector>
|
||||
#include <string>
|
||||
#include <map>
|
||||
#include <experimental/optional>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -112,25 +114,51 @@ struct DictionaryAttribute final
|
||||
const bool injective;
|
||||
};
|
||||
|
||||
struct DictionarySpecialAttribute final
|
||||
{
|
||||
const std::string name;
|
||||
const std::string expression;
|
||||
|
||||
DictionarySpecialAttribute(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
: name{config.getString(config_prefix + ".name", "")},
|
||||
expression{config.getString(config_prefix + ".expression", "")}
|
||||
{
|
||||
if (name.empty() && !expression.empty())
|
||||
throw Exception{
|
||||
"Element " + config_prefix + ".name is empty",
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
};
|
||||
}
|
||||
};
|
||||
|
||||
/// Name of identifier plus list of attributes
|
||||
struct DictionaryStructure final
|
||||
{
|
||||
std::string id_name;
|
||||
DictionarySpecialAttribute id;
|
||||
std::vector<DictionaryAttribute> attributes;
|
||||
std::string range_min;
|
||||
std::string range_max;
|
||||
std::experimental::optional<DictionarySpecialAttribute> range_min;
|
||||
std::experimental::optional<DictionarySpecialAttribute> range_max;
|
||||
bool has_expressions = false;
|
||||
|
||||
DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
: id_name{config.getString(config_prefix + ".id.name")},
|
||||
range_min{config.getString(config_prefix + ".range_min.name", "")},
|
||||
range_max{config.getString(config_prefix + ".range_max.name", "")}
|
||||
: id{config, config_prefix + ".id"}
|
||||
{
|
||||
if (id_name.empty())
|
||||
if (id.name.empty())
|
||||
throw Exception{
|
||||
"No 'id' specified for dictionary",
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
};
|
||||
|
||||
if (config.has(config_prefix + ".range_min"))
|
||||
range_min.emplace(config, config_prefix + ".range_min");
|
||||
|
||||
if (config.has(config_prefix + ".range_max"))
|
||||
range_max.emplace(config, config_prefix + ".range_max");
|
||||
|
||||
if (!id.expression.empty() ||
|
||||
(range_min && !range_min->expression.empty()) || (range_max && !range_max->expression.empty()))
|
||||
has_expressions = true;
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(config_prefix, keys);
|
||||
auto has_hierarchy = false;
|
||||
@ -148,6 +176,8 @@ struct DictionaryStructure final
|
||||
const auto underlying_type = getAttributeUnderlyingType(type_string);
|
||||
|
||||
const auto expression = config.getString(prefix + "expression", "");
|
||||
if (!expression.empty())
|
||||
has_expressions = true;
|
||||
|
||||
const auto null_value_string = config.getString(prefix + "null_value");
|
||||
Field null_value;
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <mysqlxx/Pool.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <strconvert/escape.h>
|
||||
#include "writeParenthesisedString.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -134,14 +136,35 @@ private:
|
||||
WriteBufferFromString out{query};
|
||||
writeString("SELECT ", out);
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id_name, out);
|
||||
if (!dict_struct.id.expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.id.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
if (!dict_struct.range_min.empty() && !dict_struct.range_max.empty())
|
||||
writeProbablyBackQuotedString(dict_struct.id.name, out);
|
||||
|
||||
if (dict_struct.range_min && dict_struct.range_max)
|
||||
{
|
||||
writeString(", ", out);
|
||||
writeProbablyBackQuotedString(dict_struct.range_min, out);
|
||||
|
||||
if (!dict_struct.range_min->expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.range_min->expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.range_min->name, out);
|
||||
|
||||
writeString(", ", out);
|
||||
writeProbablyBackQuotedString(dict_struct.range_max, out);
|
||||
|
||||
if (!dict_struct.range_max->expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.range_max->expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.range_max->name, out);
|
||||
}
|
||||
|
||||
for (const auto & attr : dict_struct.attributes)
|
||||
@ -150,7 +173,7 @@ private:
|
||||
|
||||
if (!attr.expression.empty())
|
||||
{
|
||||
writeString(attr.expression, out);
|
||||
writeParenthesisedString(attr.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
@ -185,7 +208,13 @@ private:
|
||||
WriteBufferFromString out{query};
|
||||
writeString("SELECT ", out);
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id_name, out);
|
||||
if (!dict_struct.id.expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.id.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id.name, out);
|
||||
|
||||
for (const auto & attr : dict_struct.attributes)
|
||||
{
|
||||
@ -193,7 +222,7 @@ private:
|
||||
|
||||
if (!attr.expression.empty())
|
||||
{
|
||||
writeString(attr.expression, out);
|
||||
writeParenthesisedString(attr.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
@ -216,7 +245,7 @@ private:
|
||||
writeString(" AND ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id_name, out);
|
||||
writeProbablyBackQuotedString(dict_struct.id.name, out);
|
||||
writeString(" IN (", out);
|
||||
|
||||
auto first = true;
|
||||
|
18
dbms/include/DB/Dictionaries/writeParenthesisedString.h
Normal file
18
dbms/include/DB/Dictionaries/writeParenthesisedString.h
Normal file
@ -0,0 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
void writeParenthesisedString(const String & s, WriteBuffer & buf)
|
||||
{
|
||||
writeChar('(', buf);
|
||||
writeString(s, buf);
|
||||
writeChar(')', buf);
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -147,6 +147,16 @@ private:
|
||||
return type_res;
|
||||
}
|
||||
|
||||
static const DataTypePtr & getScalarType(const DataTypePtr & type)
|
||||
{
|
||||
const auto array = typeid_cast<const DataTypeArray *>(type.get());
|
||||
|
||||
if (!array)
|
||||
return type;
|
||||
|
||||
return getScalarType(array->getNestedType());
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
@ -176,12 +186,16 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
/// Все аргументы должны быть константами.
|
||||
for (size_t i = 0, size = arguments.size(); i < size; ++i)
|
||||
if (!block.getByPosition(arguments[i]).column->isConst())
|
||||
throw Exception("Arguments for function array must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
const auto is_const = [&] {
|
||||
for (const auto arg_num : arguments)
|
||||
if (!block.getByPosition(arg_num).column->isConst())
|
||||
return false;
|
||||
|
||||
DataTypePtr result_type = block.getByPosition(arguments[0]).type;
|
||||
return true;
|
||||
}();
|
||||
|
||||
const auto first_arg = block.getByPosition(arguments[0]);
|
||||
DataTypePtr result_type = first_arg.type;
|
||||
if (result_type->behavesAsNumber())
|
||||
{
|
||||
/// Если тип числовой, вычисляем наименьший общий тип
|
||||
@ -189,16 +203,43 @@ public:
|
||||
result_type = getLeastCommonType(result_type, block.getByPosition(arguments[i]).type);
|
||||
}
|
||||
|
||||
Array arr;
|
||||
for (size_t i = 0, size = arguments.size(); i < size; ++i)
|
||||
if (block.getByPosition(arguments[i]).type->getName() == result_type->getName())
|
||||
/// Если элемент такого же типа как результат, просто добавляем его в ответ
|
||||
arr.push_back((*block.getByPosition(arguments[i]).column)[0]);
|
||||
else
|
||||
/// Иначе необходимо привести его к типу результата
|
||||
addField(result_type, (*block.getByPosition(arguments[i]).column)[0], arr);
|
||||
if (is_const)
|
||||
{
|
||||
Array arr;
|
||||
for (const auto arg_num : arguments)
|
||||
if (block.getByPosition(arg_num).type->getName() == result_type->getName())
|
||||
/// Если элемент такого же типа как результат, просто добавляем его в ответ
|
||||
arr.push_back((*block.getByPosition(arg_num).column)[0]);
|
||||
else
|
||||
/// Иначе необходимо привести его к типу результата
|
||||
addField(result_type, (*block.getByPosition(arg_num).column)[0], arr);
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstArray(block.getByPosition(arguments[0]).column->size(), arr, new DataTypeArray(result_type));
|
||||
block.getByPosition(result).column = new ColumnConstArray{
|
||||
first_arg.column->size(), arr, new DataTypeArray{result_type}
|
||||
};
|
||||
}
|
||||
else
|
||||
{
|
||||
auto out = new ColumnArray{result_type->createColumn()};
|
||||
ColumnPtr out_ptr{out};
|
||||
|
||||
for (const auto row_num : ext::range(0, first_arg.column->size()))
|
||||
{
|
||||
Array arr;
|
||||
|
||||
for (const auto arg_num : arguments)
|
||||
if (block.getByPosition(arg_num).type->getName() == result_type->getName())
|
||||
/// Если элемент такого же типа как результат, просто добавляем его в ответ
|
||||
arr.push_back((*block.getByPosition(arg_num).column)[row_num]);
|
||||
else
|
||||
/// Иначе необходимо привести его к типу результата
|
||||
addField(result_type, (*block.getByPosition(arg_num).column)[row_num], arr);
|
||||
|
||||
out->insert(arr);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = out_ptr;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
@ -735,7 +776,7 @@ public:
|
||||
struct IndexToOne
|
||||
{
|
||||
typedef UInt8 ResultType;
|
||||
static inline bool apply(size_t j, ResultType & current) { current = 1; return false; }
|
||||
static bool apply(size_t j, ResultType & current) { current = 1; return false; }
|
||||
};
|
||||
|
||||
/// Для indexOf.
|
||||
@ -743,23 +784,29 @@ struct IndexIdentity
|
||||
{
|
||||
typedef UInt64 ResultType;
|
||||
/// Индекс возвращается начиная с единицы.
|
||||
static inline bool apply(size_t j, ResultType & current) { current = j + 1; return false; }
|
||||
static bool apply(size_t j, ResultType & current) { current = j + 1; return false; }
|
||||
};
|
||||
|
||||
/// Для countEqual.
|
||||
struct IndexCount
|
||||
{
|
||||
typedef UInt32 ResultType;
|
||||
static inline bool apply(size_t j, ResultType & current) { ++current; return true; }
|
||||
static bool apply(size_t j, ResultType & current) { ++current; return true; }
|
||||
};
|
||||
|
||||
|
||||
template <typename T, typename IndexConv>
|
||||
struct ArrayIndexNumImpl
|
||||
{
|
||||
/// compares `lhs` against `i`-th element of `rhs`
|
||||
static bool compare(const T & lhs, const PODArray<T> & rhs, const std::size_t i ) { return lhs == rhs[i]; }
|
||||
/// compares `lhs against `rhs`, third argument unused
|
||||
static bool compare(const T & lhs, const T & rhs, std::size_t) { return lhs == rhs; }
|
||||
|
||||
template <typename ScalarOrVector>
|
||||
static void vector(
|
||||
const PODArray<T> & data, const ColumnArray::Offsets_t & offsets,
|
||||
const T value,
|
||||
const ScalarOrVector & value,
|
||||
PODArray<typename IndexConv::ResultType> & result)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
@ -773,7 +820,7 @@ struct ArrayIndexNumImpl
|
||||
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
if (data[current_offset + j] == value)
|
||||
if (compare(data[current_offset + j], value, i))
|
||||
{
|
||||
if (!IndexConv::apply(j, current))
|
||||
break;
|
||||
@ -789,19 +836,19 @@ struct ArrayIndexNumImpl
|
||||
template <typename IndexConv>
|
||||
struct ArrayIndexStringImpl
|
||||
{
|
||||
static void vector(
|
||||
static void vector_const(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
|
||||
const String & value,
|
||||
PODArray<typename IndexConv::ResultType> & result)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
size_t value_size = value.size();
|
||||
const auto size = offsets.size();
|
||||
const auto value_size = value.size();
|
||||
result.resize(size);
|
||||
|
||||
ColumnArray::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
const auto array_size = offsets[i] - current_offset;
|
||||
typename IndexConv::ResultType current = 0;
|
||||
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
@ -823,6 +870,42 @@ struct ArrayIndexStringImpl
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_vector(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
|
||||
const ColumnString::Chars_t & item_values, const ColumnString::Offsets_t & item_offsets,
|
||||
PODArray<typename IndexConv::ResultType> & result)
|
||||
{
|
||||
const auto size = offsets.size();
|
||||
result.resize(size);
|
||||
|
||||
ColumnArray::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const auto array_size = offsets[i] - current_offset;
|
||||
typename IndexConv::ResultType current = 0;
|
||||
const auto value_pos = 0 == i ? 0 : item_offsets[i - 1];
|
||||
const auto value_size = item_offsets[i] - value_pos;
|
||||
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
ColumnArray::Offset_t string_pos = current_offset == 0 && j == 0
|
||||
? 0
|
||||
: string_offsets[current_offset + j - 1];
|
||||
|
||||
ColumnArray::Offset_t string_size = string_offsets[current_offset + j] - string_pos;
|
||||
|
||||
if (string_size == value_size && 0 == memcmp(&item_values[value_pos], &data[string_pos], value_size))
|
||||
{
|
||||
if (!IndexConv::apply(j, current))
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
result[i] = current;
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -837,7 +920,7 @@ private:
|
||||
typedef ColumnVector<typename IndexConv::ResultType> ResultColumnType;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const Field & value)
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -849,19 +932,31 @@ private:
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
ResultColumnType * col_res = new ResultColumnType;
|
||||
block.getByPosition(result).column = col_res;
|
||||
const auto item_arg = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
ArrayIndexNumImpl<T, IndexConv>::vector(
|
||||
col_nested->getData(),
|
||||
col_array->getOffsets(),
|
||||
safeGet<typename NearestFieldType<T>::Type>(value),
|
||||
col_res->getData());
|
||||
if (const auto item_arg_const = typeid_cast<const ColumnConst<T> *>(item_arg))
|
||||
{
|
||||
const auto col_res = new ResultColumnType;
|
||||
ColumnPtr col_ptr{col_res};
|
||||
block.getByPosition(result).column = col_ptr;
|
||||
|
||||
ArrayIndexNumImpl<T, IndexConv>::vector(col_nested->getData(), col_array->getOffsets(),
|
||||
item_arg_const->getData(), col_res->getData());
|
||||
}
|
||||
else if (const auto item_arg_vector = typeid_cast<const ColumnVector<T> *>(item_arg))
|
||||
{
|
||||
const auto col_res = new ResultColumnType;
|
||||
ColumnPtr col_ptr{col_res};
|
||||
block.getByPosition(result).column = col_ptr;
|
||||
|
||||
ArrayIndexNumImpl<T, IndexConv>::vector(col_nested->getData(), col_array->getOffsets(),
|
||||
item_arg_vector->getData(), col_res->getData());
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const Field & value)
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -873,20 +968,32 @@ private:
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
ResultColumnType * col_res = new ResultColumnType;
|
||||
block.getByPosition(result).column = col_res;
|
||||
const auto item_arg = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
ArrayIndexStringImpl<IndexConv>::vector(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
safeGet<const String &>(value),
|
||||
col_res->getData());
|
||||
if (const auto item_arg_const = typeid_cast<const ColumnConst<String> *>(item_arg))
|
||||
{
|
||||
const auto col_res = new ResultColumnType;
|
||||
ColumnPtr col_ptr{col_res};
|
||||
block.getByPosition(result).column = col_ptr;
|
||||
|
||||
ArrayIndexStringImpl<IndexConv>::vector_const(col_nested->getChars(), col_array->getOffsets(),
|
||||
col_nested->getOffsets(), item_arg_const->getData(), col_res->getData());
|
||||
}
|
||||
else if (const auto item_arg_vector = typeid_cast<const ColumnString *>(item_arg))
|
||||
{
|
||||
const auto col_res = new ResultColumnType;
|
||||
ColumnPtr col_ptr{col_res};
|
||||
block.getByPosition(result).column = col_ptr;
|
||||
|
||||
ArrayIndexStringImpl<IndexConv>::vector_vector(col_nested->getChars(), col_array->getOffsets(),
|
||||
col_nested->getOffsets(), item_arg_vector->getChars(), item_arg_vector->getOffsets(),
|
||||
col_res->getData());
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & value)
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnConstArray * col_array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -895,23 +1002,48 @@ private:
|
||||
|
||||
const Array & arr = col_array->getData();
|
||||
|
||||
size_t i = 0;
|
||||
size_t size = arr.size();
|
||||
typename IndexConv::ResultType current = 0;
|
||||
|
||||
for (; i < size; ++i)
|
||||
const auto item_arg = block.getByPosition(arguments[1]).column.get();
|
||||
if (item_arg->isConst())
|
||||
{
|
||||
if (arr[i] == value)
|
||||
typename IndexConv::ResultType current{};
|
||||
const auto & value = (*item_arg)[0];
|
||||
|
||||
for (size_t i = 0, size = arr.size(); i < size; ++i)
|
||||
{
|
||||
if (!IndexConv::apply(i, current))
|
||||
break;
|
||||
if (arr[i] == value)
|
||||
{
|
||||
if (!IndexConv::apply(i, current))
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
|
||||
item_arg->size(),
|
||||
static_cast<typename NearestFieldType<typename IndexConv::ResultType>::Type>(current));
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto size = item_arg->size();
|
||||
const auto col_res = new ResultColumnType{size, {}};
|
||||
ColumnPtr col_ptr{col_res};
|
||||
block.getByPosition(result).column = col_ptr;
|
||||
|
||||
auto & data = col_res->getData();
|
||||
|
||||
for (size_t row = 0; row < size; ++row)
|
||||
{
|
||||
const auto & value = (*item_arg)[row];
|
||||
for (size_t i = 0, size = arr.size(); i < size; ++i)
|
||||
{
|
||||
if (arr[i] == value)
|
||||
{
|
||||
if (!IndexConv::apply(i, data[row]))
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
|
||||
block.rowsInFirstColumn(),
|
||||
static_cast<typename NearestFieldType<typename IndexConv::ResultType>::Type>(current));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -945,26 +1077,23 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (!block.getByPosition(arguments[1]).column->isConst())
|
||||
throw Exception("Second argument for function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
Field value = (*block.getByPosition(arguments[1]).column)[0];
|
||||
|
||||
if (!( executeNumber<UInt8> (block, arguments, result, value)
|
||||
|| executeNumber<UInt16> (block, arguments, result, value)
|
||||
|| executeNumber<UInt32> (block, arguments, result, value)
|
||||
|| executeNumber<UInt64> (block, arguments, result, value)
|
||||
|| executeNumber<Int8> (block, arguments, result, value)
|
||||
|| executeNumber<Int16> (block, arguments, result, value)
|
||||
|| executeNumber<Int32> (block, arguments, result, value)
|
||||
|| executeNumber<Int64> (block, arguments, result, value)
|
||||
|| executeNumber<Float32> (block, arguments, result, value)
|
||||
|| executeNumber<Float64> (block, arguments, result, value)
|
||||
|| executeConst (block, arguments, result, value)
|
||||
|| executeString (block, arguments, result, value)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (!(executeNumber<UInt8>(block, arguments, result)
|
||||
|| executeNumber<UInt16>(block, arguments, result)
|
||||
|| executeNumber<UInt32>(block, arguments, result)
|
||||
|| executeNumber<UInt64>(block, arguments, result)
|
||||
|| executeNumber<Int8>(block, arguments, result)
|
||||
|| executeNumber<Int16>(block, arguments, result)
|
||||
|| executeNumber<Int32>(block, arguments, result)
|
||||
|| executeNumber<Int64>(block, arguments, result)
|
||||
|| executeNumber<Float32>(block, arguments, result)
|
||||
|| executeNumber<Float64>(block, arguments, result)
|
||||
|| executeConst(block, arguments, result)
|
||||
|| executeString(block, arguments, result)))
|
||||
throw Exception{
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -42,31 +42,9 @@ namespace DB
|
||||
const auto ipv4_bytes_length = 4;
|
||||
const auto ipv6_bytes_length = 16;
|
||||
|
||||
class FunctionIPv6NumToString : public IFunction
|
||||
class IPv6Format
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "IPv6NumToString";
|
||||
static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; }
|
||||
|
||||
String getName() const { return name; }
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const auto ptr = typeid_cast<const DataTypeFixedString *>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
private:
|
||||
/// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base)
|
||||
static constexpr uint32_t int_log(const uint32_t value, const uint32_t base, const bool carry = false)
|
||||
{
|
||||
@ -99,23 +77,25 @@ public:
|
||||
}
|
||||
|
||||
/// print IPv4 address as %u.%u.%u.%u
|
||||
static void ipv4_format(const unsigned char * src, char *& dst)
|
||||
static void ipv4_format(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count)
|
||||
{
|
||||
constexpr auto size = sizeof(UInt32);
|
||||
const auto limit = ipv4_bytes_length - zeroed_tail_bytes_count;
|
||||
|
||||
for (const auto i : ext::range(0, size))
|
||||
for (const auto i : ext::range(0, ipv4_bytes_length))
|
||||
{
|
||||
print_integer<10, UInt8>(dst, src[i]);
|
||||
UInt8 byte = (i < limit) ? src[i] : 0;
|
||||
print_integer<10, UInt8>(dst, byte);
|
||||
|
||||
if (i != size - 1)
|
||||
if (i != ipv4_bytes_length - 1)
|
||||
*dst++ = '.';
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
/** rewritten inet_ntop6 from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c
|
||||
* performs significantly faster than the reference implementation due to the absence of sprintf calls,
|
||||
* bounds checking, unnecessary string copying and length calculation */
|
||||
static const void ipv6_format(const unsigned char * src, char *& dst)
|
||||
static const void apply(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count = 0)
|
||||
{
|
||||
struct { int base, len; } best{-1}, cur{-1};
|
||||
std::array<uint16_t, ipv6_bytes_length / sizeof(uint16_t)> words{};
|
||||
@ -123,7 +103,7 @@ public:
|
||||
/** Preprocess:
|
||||
* Copy the input (bytewise) array into a wordwise array.
|
||||
* Find the longest run of 0x00's in src[] for :: shorthanding. */
|
||||
for (const auto i : ext::range(0, ipv6_bytes_length))
|
||||
for (const auto i : ext::range(0, ipv6_bytes_length - zeroed_tail_bytes_count))
|
||||
words[i / 2] |= src[i] << ((1 - (i % 2)) << 3);
|
||||
|
||||
for (const auto i : ext::range(0, words.size()))
|
||||
@ -172,7 +152,7 @@ public:
|
||||
/// Is this address an encapsulated IPv4?
|
||||
if (i == 6 && best.base == 0 && (best.len == 6 || (best.len == 5 && words[5] == 0xffffu)))
|
||||
{
|
||||
ipv4_format(src + 12, dst);
|
||||
ipv4_format(src + 12, dst, std::min(zeroed_tail_bytes_count, static_cast<UInt8>(ipv4_bytes_length)));
|
||||
break;
|
||||
}
|
||||
|
||||
@ -185,6 +165,32 @@ public:
|
||||
|
||||
*dst++ = '\0';
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionIPv6NumToString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "IPv6NumToString";
|
||||
static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; }
|
||||
|
||||
String getName() const { return name; }
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const auto ptr = typeid_cast<const DataTypeFixedString *>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
@ -216,7 +222,7 @@ public:
|
||||
|
||||
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i)
|
||||
{
|
||||
ipv6_format(&vec_in[offset], pos);
|
||||
IPv6Format::apply(&vec_in[offset], pos);
|
||||
offsets_res[i] = pos - begin;
|
||||
}
|
||||
|
||||
@ -236,7 +242,7 @@ public:
|
||||
|
||||
char buf[INET6_ADDRSTRLEN];
|
||||
char * dst = buf;
|
||||
ipv6_format(reinterpret_cast<const unsigned char *>(data_in.data()), dst);
|
||||
IPv6Format::apply(reinterpret_cast<const unsigned char *>(data_in.data()), dst);
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstString{col_in->size(), buf};
|
||||
}
|
||||
@ -247,6 +253,172 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionCutIPv6 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "cutIPv6";
|
||||
static IFunction * create(const Context & context) { return new FunctionCutIPv6; }
|
||||
|
||||
String getName() const { return name; }
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 3)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 3.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const auto ptr = typeid_cast<const DataTypeFixedString *>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument 1 of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!typeid_cast<const DataTypeUInt8 *>(arguments[1].get()))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() +
|
||||
" of argument 2 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!typeid_cast<const DataTypeUInt8 *>(arguments[2].get()))
|
||||
throw Exception("Illegal type " + arguments[2]->getName() +
|
||||
" of argument 3 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
const auto & col_name_type = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_name_type.column;
|
||||
|
||||
const auto & col_ipv6_zeroed_tail_bytes_type = block.getByPosition(arguments[1]);
|
||||
const auto & col_ipv6_zeroed_tail_bytes = col_ipv6_zeroed_tail_bytes_type.column;
|
||||
const auto & col_ipv4_zeroed_tail_bytes_type = block.getByPosition(arguments[2]);
|
||||
const auto & col_ipv4_zeroed_tail_bytes = col_ipv4_zeroed_tail_bytes_type.column;
|
||||
|
||||
if (const auto col_in = typeid_cast<const ColumnFixedString *>(column.get()))
|
||||
{
|
||||
if (col_in->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_name_type.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto ipv6_zeroed_tail_bytes = typeid_cast<const ColumnConst<UInt8> *>(col_ipv6_zeroed_tail_bytes.get());
|
||||
if (!ipv6_zeroed_tail_bytes)
|
||||
throw Exception("Illegal type " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||
" of argument 2 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getData();
|
||||
if (ipv6_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||
throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||
" of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto ipv4_zeroed_tail_bytes = typeid_cast<const ColumnConst<UInt8> *>(col_ipv4_zeroed_tail_bytes.get());
|
||||
if (!ipv4_zeroed_tail_bytes)
|
||||
throw Exception("Illegal type " + col_ipv4_zeroed_tail_bytes_type.type->getName() +
|
||||
" of argument 3 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getData();
|
||||
if (ipv4_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||
throw Exception("Illegal value for argument 3 " + col_ipv4_zeroed_tail_bytes_type.type->getName() +
|
||||
" of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto size = col_in->size();
|
||||
const auto & vec_in = col_in->getChars();
|
||||
|
||||
auto col_res = new ColumnString;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnString::Chars_t & vec_res = col_res->getChars();
|
||||
ColumnString::Offsets_t & offsets_res = col_res->getOffsets();
|
||||
vec_res.resize(size * INET6_ADDRSTRLEN);
|
||||
offsets_res.resize(size);
|
||||
|
||||
auto begin = reinterpret_cast<char *>(&vec_res[0]);
|
||||
auto pos = begin;
|
||||
|
||||
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i)
|
||||
{
|
||||
const auto address = &vec_in[offset];
|
||||
UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count;
|
||||
cutAddress(address, pos, zeroed_tail_bytes_count);
|
||||
offsets_res[i] = pos - begin;
|
||||
}
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
}
|
||||
else if (const auto col_in = typeid_cast<const ColumnConst<String> *>(column.get()))
|
||||
{
|
||||
const auto data_type_fixed_string = typeid_cast<const DataTypeFixedString *>(col_in->getDataType().get());
|
||||
if (!data_type_fixed_string || data_type_fixed_string->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_name_type.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto ipv6_zeroed_tail_bytes = typeid_cast<const ColumnConst<UInt8> *>(col_ipv6_zeroed_tail_bytes.get());
|
||||
if (!ipv6_zeroed_tail_bytes)
|
||||
throw Exception("Illegal type " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||
" of argument 2 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getData();
|
||||
if (ipv6_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||
throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||
" of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto ipv4_zeroed_tail_bytes = typeid_cast<const ColumnConst<UInt8> *>(col_ipv4_zeroed_tail_bytes.get());
|
||||
if (!ipv4_zeroed_tail_bytes)
|
||||
throw Exception("Illegal type " + col_ipv4_zeroed_tail_bytes_type.type->getName() +
|
||||
" of argument 3 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getData();
|
||||
if (ipv4_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||
throw Exception("Illegal value for argument 3 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||
" of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto & data_in = col_in->getData();
|
||||
|
||||
char buf[INET6_ADDRSTRLEN];
|
||||
char * dst = buf;
|
||||
|
||||
const auto address = reinterpret_cast<const unsigned char *>(data_in.data());
|
||||
UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count;
|
||||
cutAddress(address, dst, zeroed_tail_bytes_count);
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstString{col_in->size(), buf};
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
bool isIPv4Mapped(const unsigned char * address) const
|
||||
{
|
||||
return (*reinterpret_cast<const UInt64 *>(&address[0]) == 0) &&
|
||||
((*reinterpret_cast<const UInt64 *>(&address[8]) & 0x00000000FFFFFFFFull) == 0x00000000FFFF0000ull);
|
||||
}
|
||||
|
||||
void cutAddress(const unsigned char * address, char *& dst, UInt8 zeroed_tail_bytes_count)
|
||||
{
|
||||
IPv6Format::apply(address, dst, zeroed_tail_bytes_count);
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionIPv6StringToNum : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -767,6 +939,69 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionIPv4ToIPv6 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "IPv4ToIPv6";
|
||||
static IFunction * create(const Context & context) { return new FunctionIPv4ToIPv6; }
|
||||
|
||||
String getName() const { return name; }
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (typeid_cast<const DataTypeUInt32 *>(arguments[0].get()) == nullptr)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeFixedString{16};
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
const auto & col_name_type = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_name_type.column;
|
||||
|
||||
if (const auto col_in = typeid_cast<const ColumnVector<UInt32> *>(column.get()))
|
||||
{
|
||||
const auto col_res = new ColumnFixedString{ipv6_bytes_length};
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
auto & vec_res = col_res->getChars();
|
||||
vec_res.resize(col_in->size() * ipv6_bytes_length);
|
||||
|
||||
const auto & vec_in = col_in->getData();
|
||||
|
||||
for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += ipv6_bytes_length, ++i)
|
||||
mapIPv4ToIPv6(vec_in[i], &vec_res[out_offset]);
|
||||
}
|
||||
else if (const auto col_in = typeid_cast<const ColumnConst<UInt32> *>(column.get()))
|
||||
{
|
||||
std::string buf;
|
||||
buf.resize(ipv6_bytes_length);
|
||||
mapIPv4ToIPv6(col_in->getData(), reinterpret_cast<unsigned char *>(&buf[0]));
|
||||
|
||||
ColumnConstString * col_res = new ColumnConstString(ipv6_bytes_length, buf,
|
||||
new DataTypeFixedString{ipv6_bytes_length});
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
void mapIPv4ToIPv6(UInt32 in, unsigned char * buf) const
|
||||
{
|
||||
*reinterpret_cast<UInt64 *>(&buf[0]) = 0;
|
||||
*reinterpret_cast<UInt64 *>(&buf[8]) = 0x00000000FFFF0000ull | (static_cast<UInt64>(ntohl(in)) << 32);
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionHex : public IFunction
|
||||
{
|
||||
@ -1635,7 +1870,8 @@ private:
|
||||
if (const auto value_col = typeid_cast<const ColumnVector<T> *>(value_col_untyped))
|
||||
{
|
||||
const auto size = value_col->size();
|
||||
const auto & mask = createMask<T>(size, block, arguments);
|
||||
bool is_const;
|
||||
const auto mask = createConstMask<T>(size, block, arguments, is_const);
|
||||
const auto & val = value_col->getData();
|
||||
|
||||
const auto out_col = new ColumnVector<UInt8>(size);
|
||||
@ -1644,25 +1880,46 @@ private:
|
||||
|
||||
auto & out = out_col->getData();
|
||||
|
||||
for (const auto i : ext::range(0, size))
|
||||
out[i] = Impl::combine(val[i], mask[i]);
|
||||
if (is_const)
|
||||
{
|
||||
for (const auto i : ext::range(0, size))
|
||||
out[i] = Impl::combine(val[i], mask);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto mask = createMask<T>(size, block, arguments);
|
||||
|
||||
for (const auto i : ext::range(0, size))
|
||||
out[i] = Impl::combine(val[i], mask[i]);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto value_col = typeid_cast<const ColumnConst<T> *>(value_col_untyped))
|
||||
{
|
||||
const auto size = value_col->size();
|
||||
const auto & mask = createMask<T>(size, block, arguments);
|
||||
const auto & val = value_col->getData();
|
||||
bool is_const;
|
||||
const auto mask = createConstMask<T>(size, block, arguments, is_const);
|
||||
const auto val = value_col->getData();
|
||||
|
||||
const auto out_col = new ColumnVector<UInt8>(size);
|
||||
ColumnPtr out_col_ptr{out_col};
|
||||
block.getByPosition(result).column = out_col_ptr;
|
||||
if (is_const)
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<UInt8>{
|
||||
size, Impl::combine(val, mask)
|
||||
};
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto mask = createMask<T>(size, block, arguments);
|
||||
const auto out_col = new ColumnVector<UInt8>(size);
|
||||
ColumnPtr out_col_ptr{out_col};
|
||||
block.getByPosition(result).column = out_col_ptr;
|
||||
|
||||
auto & out = out_col->getData();
|
||||
auto & out = out_col->getData();
|
||||
|
||||
for (const auto i : ext::range(0, size))
|
||||
out[i] = Impl::combine(val, mask[i]);
|
||||
for (const auto i : ext::range(0, size))
|
||||
out[i] = Impl::combine(val, mask[i]);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -1670,26 +1927,49 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
PODArray<T> createMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments)
|
||||
template <typename ValueType>
|
||||
ValueType createConstMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments, bool & is_const)
|
||||
{
|
||||
PODArray<T> mask(size, T{});
|
||||
is_const = true;
|
||||
ValueType mask{};
|
||||
|
||||
for (const auto i : ext::range(1, arguments.size()))
|
||||
addToMask(mask, block.getByPosition(arguments[i]).column.get());
|
||||
{
|
||||
const auto pos_col = block.getByPosition(arguments[i]).column.get();
|
||||
|
||||
if (pos_col->isConst())
|
||||
{
|
||||
const auto pos = static_cast<const ColumnConst<ValueType> *>(pos_col)->getData();
|
||||
mask = mask | 1 << pos;
|
||||
}
|
||||
else
|
||||
{
|
||||
is_const = false;
|
||||
return {};
|
||||
}
|
||||
}
|
||||
|
||||
return mask;
|
||||
}
|
||||
|
||||
template <typename ValueType>
|
||||
void addToMask(PODArray<ValueType> & mask, const IColumn * const pos_col)
|
||||
PODArray<ValueType> createMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments)
|
||||
{
|
||||
if (!addToMaskImpl<UInt8>(mask, pos_col) && !addToMaskImpl<UInt16>(mask, pos_col) &&
|
||||
!addToMaskImpl<UInt32>(mask, pos_col) && !addToMaskImpl<UInt64>(mask, pos_col))
|
||||
throw Exception{
|
||||
"Illegal column " + pos_col->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
PODArray<ValueType> mask(size, ValueType{});
|
||||
|
||||
for (const auto i : ext::range(1, arguments.size()))
|
||||
{
|
||||
const auto pos_col = block.getByPosition(arguments[i]).column.get();
|
||||
|
||||
if (!addToMaskImpl<UInt8>(mask, pos_col) && !addToMaskImpl<UInt16>(mask, pos_col) &&
|
||||
!addToMaskImpl<UInt32>(mask, pos_col) && !addToMaskImpl<UInt64>(mask, pos_col))
|
||||
throw Exception{
|
||||
"Illegal column " + pos_col->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
|
||||
return mask;
|
||||
}
|
||||
|
||||
template <typename PosType, typename ValueType>
|
||||
|
@ -422,8 +422,8 @@ public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() < 2)
|
||||
throw Exception("Function " + getName() + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
if (arguments.size() < 1)
|
||||
throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new DataTypeTuple(arguments);
|
||||
}
|
||||
|
@ -24,6 +24,8 @@ namespace DB
|
||||
* rand - linear congruental generator 0 .. 2^32 - 1.
|
||||
* rand64 - комбинирует несколько значений rand, чтобы получить значения из диапазона 0 .. 2^64 - 1.
|
||||
*
|
||||
* randConstant - служебная функция, выдаёт константный столбец со случайным значением.
|
||||
*
|
||||
* В качестве затравки используют время.
|
||||
* Замечание: переинициализируется на каждый блок.
|
||||
* Это значит, что таймер должен быть достаточного разрешения, чтобы выдавать разные значения на каждый блок.
|
||||
@ -182,11 +184,60 @@ public:
|
||||
};
|
||||
|
||||
|
||||
struct NameRand { static constexpr auto name = "rand"; };
|
||||
struct NameRand64 { static constexpr auto name = "rand64"; };
|
||||
template <typename Impl, typename Name>
|
||||
class FunctionRandomConstant : public IFunction
|
||||
{
|
||||
private:
|
||||
typedef typename Impl::ReturnType ToType;
|
||||
|
||||
/// Значение одно для разных блоков.
|
||||
bool is_initialized = false;
|
||||
ToType value;
|
||||
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static IFunction * create(const Context & context) { return new FunctionRandomConstant; }
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() > 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 0 or 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new typename DataTypeFromFieldType<typename Impl::ReturnType>::Type;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (!is_initialized)
|
||||
{
|
||||
is_initialized = true;
|
||||
typename ColumnVector<ToType>::Container_t vec_to(1);
|
||||
Impl::execute(vec_to);
|
||||
value = vec_to[0];
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = new ColumnConst<ToType>(block.rowsInFirstColumn(), value);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NameRand { static constexpr auto name = "rand"; };
|
||||
struct NameRand64 { static constexpr auto name = "rand64"; };
|
||||
struct NameRandConstant { static constexpr auto name = "randConstant"; };
|
||||
|
||||
typedef FunctionRandom<RandImpl, NameRand> FunctionRand;
|
||||
typedef FunctionRandom<Rand64Impl, NameRand64> FunctionRand64;
|
||||
typedef FunctionRandomConstant<RandImpl, NameRandConstant> FunctionRandConstant;
|
||||
|
||||
|
||||
}
|
||||
|
@ -323,8 +323,13 @@ namespace Regexps
|
||||
if (known_regexps.end() == it)
|
||||
it = known_regexps.emplace(pattern, std::make_unique<Holder>()).first;
|
||||
|
||||
return it->second->get([&pattern] {
|
||||
return new Regexp{createRegexp<like>(pattern, no_capture ? OptimizedRegularExpression::RE_NO_CAPTURE : 0)};
|
||||
return it->second->get([&pattern]
|
||||
{
|
||||
int flags = OptimizedRegularExpression::RE_DOT_NL;
|
||||
if (no_capture)
|
||||
flags |= OptimizedRegularExpression::RE_NO_CAPTURE;
|
||||
|
||||
return new Regexp{createRegexp<like>(pattern, flags)};
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -59,22 +59,27 @@ struct Memory : boost::noncopyable, Allocator
|
||||
|
||||
void resize(size_t new_size)
|
||||
{
|
||||
if (new_size < m_capacity)
|
||||
if (0 == m_capacity)
|
||||
{
|
||||
m_size = m_capacity = new_size;
|
||||
alloc();
|
||||
}
|
||||
else if (new_size < m_capacity)
|
||||
{
|
||||
m_size = new_size;
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
new_size = align(new_size);
|
||||
new_size = align(new_size, alignment);
|
||||
/// @todo pointer to void can be converted to pointer to any type with static_cast by ISO C++, reinterpret_cast has no advantages
|
||||
m_data = reinterpret_cast<char *>(Allocator::realloc(m_data, m_capacity, new_size, alignment));
|
||||
m_capacity = new_size;
|
||||
m_size = m_capacity;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
size_t align(size_t value) const
|
||||
static size_t align(const size_t value, const size_t alignment)
|
||||
{
|
||||
if (!alignment)
|
||||
return value;
|
||||
@ -82,6 +87,7 @@ private:
|
||||
return (value + alignment - 1) / alignment * alignment;
|
||||
}
|
||||
|
||||
private:
|
||||
void alloc()
|
||||
{
|
||||
if (!m_capacity)
|
||||
@ -93,7 +99,8 @@ private:
|
||||
ProfileEvents::increment(ProfileEvents::IOBufferAllocs);
|
||||
ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, m_capacity);
|
||||
|
||||
size_t new_capacity = align(m_capacity);
|
||||
size_t new_capacity = align(m_capacity, alignment);
|
||||
/// @todo pointer to void can be converted to pointer to any type with static_cast by ISO C++, reinterpret_cast has no advantages
|
||||
m_data = reinterpret_cast<char *>(Allocator::alloc(new_capacity, alignment));
|
||||
m_capacity = new_capacity;
|
||||
m_size = m_capacity;
|
||||
@ -104,6 +111,7 @@ private:
|
||||
if (!m_data)
|
||||
return;
|
||||
|
||||
/// @todo pointer to any type can be implicitly converted to pointer to void, no cast required
|
||||
Allocator::free(reinterpret_cast<void *>(m_data), m_capacity);
|
||||
m_data = nullptr; /// Чтобы избежать double free, если последующий вызов alloc кинет исключение.
|
||||
}
|
||||
|
@ -82,10 +82,11 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
CachedCompressedReadBuffer(const std::string & path_, UncompressedCache * cache_, size_t estimated_size_,
|
||||
size_t aio_threshold_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_),
|
||||
estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0)
|
||||
CachedCompressedReadBuffer(
|
||||
const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_,
|
||||
size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_), estimated_size(estimated_size_),
|
||||
aio_threshold(aio_threshold_), file_pos(0)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -42,10 +42,11 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
CompressedReadBufferFromFile(const std::string & path, size_t estimated_size, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
CompressedReadBufferFromFile(
|
||||
const std::string & path, size_t estimated_size, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: BufferWithOwnMemory<ReadBuffer>(0),
|
||||
p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size)),
|
||||
file_in(*p_file_in)
|
||||
p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size)),
|
||||
file_in(*p_file_in)
|
||||
{
|
||||
compressed_in = &file_in;
|
||||
}
|
||||
|
@ -731,6 +731,11 @@ public:
|
||||
*/
|
||||
Block mergeBlocks(BlocksList & blocks, bool final);
|
||||
|
||||
/** Преобразовать (разрезать) блок частично-агрегированных данных на много блоков, как если бы использовался двухуровневый метод агрегации.
|
||||
* Это нужно, чтобы потом было проще объединить результат с другими результатами, уже являющимися двухуровневыми.
|
||||
*/
|
||||
std::vector<Block> convertBlockToTwoLevel(const Block & block);
|
||||
|
||||
using CancellationHook = std::function<bool()>;
|
||||
|
||||
/** Установить функцию, которая проверяет, можно ли прервать текущую задачу.
|
||||
@ -806,7 +811,7 @@ protected:
|
||||
/** Если заданы только имена столбцов (key_names, а также aggregates[i].column_name), то вычислить номера столбцов.
|
||||
* Сформировать блок - пример результата.
|
||||
*/
|
||||
void initialize(Block & block);
|
||||
void initialize(const Block & block);
|
||||
|
||||
/** Выбрать способ агрегации на основе количества и типов ключей. */
|
||||
AggregatedDataVariants::Type chooseAggregationMethod(const ConstColumnPlainPtrs & key_columns, Sizes & key_sizes);
|
||||
@ -961,6 +966,16 @@ protected:
|
||||
Block & block,
|
||||
AggregatedDataVariants & result) const;
|
||||
|
||||
template <typename Method>
|
||||
void convertBlockToTwoLevelImpl(
|
||||
Method & method,
|
||||
Arena * pool,
|
||||
ConstColumnPlainPtrs & key_columns,
|
||||
const Sizes & key_sizes,
|
||||
StringRefs & keys,
|
||||
const Block & source,
|
||||
std::vector<Block> & destinations) const;
|
||||
|
||||
template <typename Method>
|
||||
void destroyImpl(
|
||||
Method & method) const;
|
||||
|
@ -44,12 +44,6 @@ class QueryLog;
|
||||
struct MergeTreeSettings;
|
||||
|
||||
|
||||
/// имя таблицы -> таблица
|
||||
typedef std::map<String, StoragePtr> Tables;
|
||||
|
||||
/// имя БД -> таблицы
|
||||
typedef std::map<String, Tables> Databases;
|
||||
|
||||
/// (имя базы данных, имя таблицы)
|
||||
typedef std::pair<String, String> DatabaseAndTableName;
|
||||
|
||||
@ -272,6 +266,8 @@ public:
|
||||
private:
|
||||
const Dictionaries & getDictionariesImpl(bool throw_on_error) const;
|
||||
const ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const;
|
||||
|
||||
StoragePtr getTableImpl(const String & database_name, const String & table_name, Exception * exception) const;
|
||||
};
|
||||
|
||||
|
||||
|
@ -163,8 +163,8 @@ private:
|
||||
* - в "левой" таблице, он будет доступен по имени expr(x), так как ещё не было выполнено действие Project.
|
||||
* Надо запомнить оба этих варианта.
|
||||
*/
|
||||
NameSet join_key_names_left_set;
|
||||
NameSet join_key_names_right_set;
|
||||
Names join_key_names_left;
|
||||
Names join_key_names_right;
|
||||
|
||||
NamesAndTypesList columns_added_by_join;
|
||||
|
||||
|
@ -107,7 +107,29 @@ public:
|
||||
ExternalDictionaries(Context & context, const bool throw_on_error)
|
||||
: context(context), log(&Logger::get("ExternalDictionaries"))
|
||||
{
|
||||
reloadImpl(throw_on_error);
|
||||
{
|
||||
/** При синхронной загрузки внешних словарей в момент выполнения запроса,
|
||||
* не нужно использовать ограничение на расход памяти запросом.
|
||||
*/
|
||||
struct TemporarilyDisableMemoryTracker
|
||||
{
|
||||
MemoryTracker * memory_tracker;
|
||||
|
||||
TemporarilyDisableMemoryTracker()
|
||||
{
|
||||
memory_tracker = current_memory_tracker;
|
||||
current_memory_tracker = nullptr;
|
||||
}
|
||||
|
||||
~TemporarilyDisableMemoryTracker()
|
||||
{
|
||||
current_memory_tracker = memory_tracker;
|
||||
}
|
||||
} temporarily_disable_memory_tracker;
|
||||
|
||||
reloadImpl(throw_on_error);
|
||||
}
|
||||
|
||||
reloading_thread = std::thread{&ExternalDictionaries::reloadPeriodically, this};
|
||||
}
|
||||
|
||||
|
308
dbms/include/DB/Interpreters/InJoinSubqueriesPreprocessor.h
Normal file
308
dbms/include/DB/Interpreters/InJoinSubqueriesPreprocessor.h
Normal file
@ -0,0 +1,308 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/Parsers/ASTFunction.h>
|
||||
#include <DB/Parsers/ASTJoin.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Storages/StorageDistributed.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
#include <deque>
|
||||
#include <unordered_map>
|
||||
#include <type_traits>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
using NameToAttributes = std::unordered_map<std::string, IAST::Attributes>;
|
||||
|
||||
const NameToAttributes name_to_attributes =
|
||||
{
|
||||
{ "in", IAST::IsIn },
|
||||
{ "notIn", IAST::IsNotIn },
|
||||
{ "globalIn", IAST::IsIn | IAST::IsGlobal },
|
||||
{ "globalNotIn", IAST::IsNotIn | IAST::IsGlobal }
|
||||
};
|
||||
|
||||
/// Из названия секции IN получить соответствующие атрибуты.
|
||||
IAST::Attributes getAttributesFromInSubqueryName(const std::string & name)
|
||||
{
|
||||
auto it = name_to_attributes.find(name);
|
||||
if (it != name_to_attributes.end())
|
||||
return it->second;
|
||||
else
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Из атрибутов составить название секции IN.
|
||||
std::string getNameFromInSubqueryAttributes(IAST::Attributes attributes)
|
||||
{
|
||||
std::string name;
|
||||
|
||||
if (attributes & IAST::IsIn)
|
||||
{
|
||||
if (attributes & IAST::IsGlobal)
|
||||
name = "globalIn";
|
||||
else
|
||||
name = "in";
|
||||
}
|
||||
else if (attributes & IAST::IsNotIn)
|
||||
{
|
||||
if (attributes & IAST::IsGlobal)
|
||||
name = "globalNotIn";
|
||||
else
|
||||
name = "notIn";
|
||||
}
|
||||
|
||||
return name;
|
||||
}
|
||||
|
||||
/// Проверить, является ли движок распределённым с количеством шардов более одного.
|
||||
template <typename TStorageDistributed>
|
||||
bool isEligibleStorageForInJoinPreprocessing(const StoragePtr & storage)
|
||||
{
|
||||
if (!storage)
|
||||
return false;
|
||||
if (!storage->isRemote())
|
||||
return false;
|
||||
|
||||
auto storage_distributed = static_cast<TStorageDistributed *>(storage.get());
|
||||
if (storage_distributed->getShardCount() < 2)
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** Этот класс предоставляет контроль над выполнением распределённых запросов внутри секций IN или JOIN.
|
||||
* Мы используем шаблон, потому что движок StorageDistributed слишком сложный, чтобы писать юнит-тесты,
|
||||
* которые бы зависели от него.
|
||||
*/
|
||||
template <typename TStorageDistributed = StorageDistributed, typename Enable = void>
|
||||
class InJoinSubqueriesPreprocessor;
|
||||
|
||||
template <typename TStorageDistributed>
|
||||
class InJoinSubqueriesPreprocessor<TStorageDistributed,
|
||||
typename std::enable_if<std::is_base_of<IStorage, TStorageDistributed>::value>::type> final
|
||||
{
|
||||
public:
|
||||
InJoinSubqueriesPreprocessor(ASTSelectQuery * select_query_,
|
||||
const Context & context_, const StoragePtr & storage_)
|
||||
: select_query(select_query_), context(context_), settings(context.getSettingsRef()), storage(storage_)
|
||||
{
|
||||
}
|
||||
|
||||
InJoinSubqueriesPreprocessor(const InJoinSubqueriesPreprocessor &) = delete;
|
||||
InJoinSubqueriesPreprocessor & operator=(const InJoinSubqueriesPreprocessor &) = delete;
|
||||
|
||||
/** В зависимости от профиля пользователя проверить наличие прав на выполнение
|
||||
* распределённых подзапросов внутри секций IN или JOIN и обработать эти подзапросы.
|
||||
*/
|
||||
void perform()
|
||||
{
|
||||
if (settings.distributed_product_mode == DistributedProductMode::ALLOW)
|
||||
{
|
||||
/// Согласно профиля пользователя распределённые подзапросы внутри секций IN и JOIN разрешены.
|
||||
/// Ничего не делаем.
|
||||
return;
|
||||
}
|
||||
|
||||
if (select_query == nullptr)
|
||||
return;
|
||||
|
||||
/// Проверить главный запрос. В секции FROM должна быть указана распределённая таблица
|
||||
/// с количеством шардов более одного. Табличные функции пропускаем.
|
||||
|
||||
if (select_query->attributes & IAST::IsPreprocessedForInJoinSubqueries)
|
||||
return;
|
||||
|
||||
if (select_query->table.isNull())
|
||||
{
|
||||
select_query->setAttributes(IAST::IsPreprocessedForInJoinSubqueries);
|
||||
return;
|
||||
}
|
||||
|
||||
if (typeid_cast<const ASTFunction *>(select_query->table.get()) != nullptr)
|
||||
{
|
||||
select_query->setAttributes(IAST::IsPreprocessedForInJoinSubqueries);
|
||||
return;
|
||||
}
|
||||
|
||||
if (!isEligibleStorageForInJoinPreprocessing<TStorageDistributed>(storage))
|
||||
{
|
||||
select_query->setAttributes(IAST::IsPreprocessedForInJoinSubqueries);
|
||||
return;
|
||||
}
|
||||
|
||||
/// Собрать информацию про все подзапросы внутри секций IN или JOIN.
|
||||
/// Обработать те подзапросы, которые распределённые.
|
||||
|
||||
std::deque<IAST *> to_preprocess;
|
||||
to_preprocess.push_back(select_query);
|
||||
|
||||
while (!to_preprocess.empty())
|
||||
{
|
||||
auto node = to_preprocess.back();
|
||||
to_preprocess.pop_back();
|
||||
|
||||
ASTFunction * function;
|
||||
ASTJoin * join;
|
||||
ASTSelectQuery * sub_select_query;
|
||||
|
||||
if ((function = typeid_cast<ASTFunction *>(node)) != nullptr)
|
||||
{
|
||||
auto attributes = getAttributesFromInSubqueryName(function->name);
|
||||
if (attributes != 0)
|
||||
{
|
||||
/// Найдена секция IN.
|
||||
node->enclosing_in_or_join = node;
|
||||
node->attributes |= attributes;
|
||||
}
|
||||
}
|
||||
else if ((join = typeid_cast<ASTJoin *>(node)) != nullptr)
|
||||
{
|
||||
/// Найдена секция JOIN.
|
||||
node->enclosing_in_or_join = node;
|
||||
node->attributes |= IAST::IsJoin;
|
||||
if (join->locality == ASTJoin::Global)
|
||||
node->attributes |= IAST::IsGlobal;
|
||||
}
|
||||
else if ((node != static_cast<IAST *>(select_query))
|
||||
&& ((sub_select_query = typeid_cast<ASTSelectQuery *>(node)) != nullptr))
|
||||
{
|
||||
++node->select_query_depth;
|
||||
if (sub_select_query->enclosing_in_or_join != nullptr)
|
||||
{
|
||||
/// Найден подзапрос внутри секции IN или JOIN.
|
||||
preprocessSubquery(*sub_select_query);
|
||||
}
|
||||
}
|
||||
|
||||
for (auto & child : node->children)
|
||||
{
|
||||
if (!(child->attributes & IAST::IsPreprocessedForInJoinSubqueries))
|
||||
{
|
||||
auto n = child.get();
|
||||
n->enclosing_in_or_join = node->enclosing_in_or_join;
|
||||
n->select_query_depth = node->select_query_depth;
|
||||
to_preprocess.push_back(n);
|
||||
}
|
||||
}
|
||||
|
||||
node->attributes |= IAST::IsPreprocessedForInJoinSubqueries;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
void preprocessSubquery(ASTSelectQuery & sub_select_query)
|
||||
{
|
||||
auto & enclosing_in_or_join = *sub_select_query.enclosing_in_or_join;
|
||||
bool is_global = enclosing_in_or_join.attributes & IAST::IsGlobal;
|
||||
|
||||
/// Если подзапрос внутри секции IN или JOIN является непосредственным потомком
|
||||
/// главного запроса и указано ключевое слово GLOBAL, то подзапрос пропускается.
|
||||
if ((sub_select_query.select_query_depth == 1) && is_global)
|
||||
return;
|
||||
|
||||
auto subquery_table_storage = getDistributedSubqueryStorage(sub_select_query);
|
||||
if (!subquery_table_storage)
|
||||
return;
|
||||
|
||||
if (settings.distributed_product_mode == DistributedProductMode::DENY)
|
||||
{
|
||||
/// Согласно профиля пользователя распределённые подзапросы внутри секций IN и JOIN запрещены.
|
||||
throw Exception("You are not allowed to perform distributed IN/JOIN subqueries",
|
||||
ErrorCodes::DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED);
|
||||
}
|
||||
else if (settings.distributed_product_mode == DistributedProductMode::GLOBAL)
|
||||
{
|
||||
/// Согласно профиля пользователя распределённые подзапросы внутри секций IN и JOIN разрешены.
|
||||
/// Преобразовать [NOT] IN в GLOBAL [NOT] IN, и JOIN в GLOBAL JOIN.
|
||||
|
||||
if (!is_global)
|
||||
{
|
||||
if (enclosing_in_or_join.attributes & IAST::IsJoin)
|
||||
{
|
||||
auto & join = static_cast<ASTJoin &>(enclosing_in_or_join);
|
||||
join.locality = ASTJoin::Global;
|
||||
}
|
||||
else if (enclosing_in_or_join.attributes & (IAST::IsIn | IAST::IsNotIn))
|
||||
{
|
||||
auto & function = static_cast<ASTFunction &>(enclosing_in_or_join);
|
||||
function.name = getNameFromInSubqueryAttributes(function.attributes | IAST::IsGlobal);
|
||||
}
|
||||
else
|
||||
throw Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
else if (settings.distributed_product_mode == DistributedProductMode::LOCAL)
|
||||
{
|
||||
/// Согласно профиля пользователя распределённые подзапросы внутри секций IN и JOIN разрешены.
|
||||
/// Преобразовать распределённую таблицу в соответствующую удалённую таблицу.
|
||||
|
||||
auto & distributed_storage = static_cast<TStorageDistributed &>(*subquery_table_storage);
|
||||
|
||||
if (sub_select_query.database.isNull())
|
||||
sub_select_query.database = new ASTIdentifier{{}, distributed_storage.getRemoteDatabaseName(),
|
||||
ASTIdentifier::Database};
|
||||
else
|
||||
{
|
||||
auto & db_name = typeid_cast<ASTIdentifier &>(*sub_select_query.database).name;
|
||||
db_name = distributed_storage.getRemoteDatabaseName();
|
||||
}
|
||||
|
||||
auto & table_name = typeid_cast<ASTIdentifier &>(*sub_select_query.table).name;
|
||||
table_name = distributed_storage.getRemoteTableName();
|
||||
}
|
||||
else
|
||||
throw Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
StoragePtr getDistributedSubqueryStorage(const ASTSelectQuery & sub_select_query) const
|
||||
{
|
||||
if (sub_select_query.table.isNull())
|
||||
return {};
|
||||
|
||||
if (typeid_cast<const ASTSelectQuery *>(sub_select_query.table.get()) != nullptr)
|
||||
return {};
|
||||
|
||||
if (typeid_cast<const ASTFunction *>(sub_select_query.table.get()) != nullptr)
|
||||
return {};
|
||||
|
||||
std::string table_name = typeid_cast<const ASTIdentifier &>(*sub_select_query.table).name;
|
||||
|
||||
/// Если база данных не указана - используем текущую базу данных.
|
||||
std::string database_name;
|
||||
if (sub_select_query.database)
|
||||
database_name = typeid_cast<const ASTIdentifier &>(*sub_select_query.database).name;
|
||||
else
|
||||
{
|
||||
if (context.tryGetTable("", table_name))
|
||||
database_name = "";
|
||||
else
|
||||
database_name = context.getCurrentDatabase();
|
||||
}
|
||||
|
||||
auto subquery_table_storage = context.tryGetTable(database_name, table_name);
|
||||
if (!subquery_table_storage)
|
||||
return {};
|
||||
|
||||
if (!isEligibleStorageForInJoinPreprocessing<TStorageDistributed>(subquery_table_storage))
|
||||
return {};
|
||||
|
||||
return subquery_table_storage;
|
||||
}
|
||||
|
||||
private:
|
||||
ASTSelectQuery * select_query;
|
||||
const Context & context;
|
||||
const Settings & settings;
|
||||
const StoragePtr & storage;
|
||||
};
|
||||
|
||||
}
|
@ -14,6 +14,7 @@
|
||||
#include <DB/Common/MemoryTracker.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/Interpreters/QueryPriorities.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -41,6 +42,9 @@ struct ProcessListElement
|
||||
|
||||
bool is_cancelled = false;
|
||||
|
||||
/// Здесь могут быть зарегистрированы временные таблицы. Изменять под mutex-ом.
|
||||
Tables temporary_tables;
|
||||
|
||||
|
||||
ProcessListElement(const String & query_, const String & user_,
|
||||
const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
@ -127,7 +131,7 @@ public:
|
||||
* Если времени не хватило - кинуть исключение.
|
||||
*/
|
||||
EntryPtr insert(const String & query_, const String & user_, const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
size_t max_memory_usage, size_t max_wait_milliseconds, bool replace_running_query, QueryPriorities::Priority priority);
|
||||
const Settings & settings);
|
||||
|
||||
/// Количество одновременно выполняющихся запросов.
|
||||
size_t size() const { return cur_size; }
|
||||
@ -144,6 +148,12 @@ public:
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
max_size = max_size_;
|
||||
}
|
||||
|
||||
/// Зарегистрировать временную таблицу. Потом её можно будет получить по query_id и по названию.
|
||||
void addTemporaryTable(ProcessListElement & elem, const String & table_name, StoragePtr storage);
|
||||
|
||||
/// Найти временную таблицу по query_id и по названию. Замечание: плохо работает, если есть разные запросы с одним query_id.
|
||||
StoragePtr tryGetTemporaryTable(const String & query_id, const String & table_name) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -119,6 +119,9 @@ struct Settings
|
||||
* (Чтобы большие запросы не вымывали кэш.) */ \
|
||||
M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \
|
||||
\
|
||||
/** Распределять чтение из MergeTree по потокам равномерно, обеспечивая стабильное среднее время исполнения каждого потока в пределах одного чтения. */ \
|
||||
M(SettingBool, merge_tree_uniform_read_distribution, false) \
|
||||
\
|
||||
/** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \
|
||||
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \
|
||||
\
|
||||
@ -148,6 +151,18 @@ struct Settings
|
||||
\
|
||||
/** Логгировать запросы и писать лог в системную таблицу. */ \
|
||||
M(SettingBool, log_queries, 0) \
|
||||
\
|
||||
/** Как выполняются распределённые подзапросы внутри секций IN или JOIN? */ \
|
||||
M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY) \
|
||||
\
|
||||
/** Схема выполнения GLOBAL-подзапросов. */ \
|
||||
M(SettingGlobalSubqueriesMethod, global_subqueries_method, GlobalSubqueriesMethod::PUSH) \
|
||||
\
|
||||
/** Максимальное количество одновременно выполняющихся запросов на одного user-а. */ \
|
||||
M(SettingUInt64, max_concurrent_queries_for_user, 0) \
|
||||
\
|
||||
/** Для запросов INSERT в реплицируемую таблицу, ждать записи на указанное число реплик и лианеризовать добавление данных. 0 - отключено. */ \
|
||||
M(SettingUInt64, insert_quorum, 0) \
|
||||
|
||||
/// Всевозможные ограничения на выполнение запроса.
|
||||
Limits limits;
|
||||
@ -177,6 +192,9 @@ struct Settings
|
||||
*/
|
||||
void setProfile(const String & profile_name, Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
/// Загрузить настройки по пути из конфига
|
||||
void loadSettingsFromConfig(const String & path, Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
/// Прочитать настройки из буфера. Они записаны как набор name-value пар, идущих подряд, заканчивающихся пустым name.
|
||||
/// Если в настройках выставлено readonly=1, то игнорировать настройки.
|
||||
void deserialize(ReadBuffer & buf);
|
||||
|
@ -555,7 +555,7 @@ struct SettingCompressionMethod
|
||||
if (s == "zstd")
|
||||
return CompressionMethod::ZSTD;
|
||||
|
||||
throw Exception("Unknown compression method: '" + s + "', must be one of 'quicklz', 'lz4', 'lz4hc', 'zstd' ", ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
|
||||
throw Exception("Unknown compression method: '" + s + "', must be one of 'quicklz', 'lz4', 'lz4hc', 'zstd'", ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
|
||||
}
|
||||
|
||||
String toString() const
|
||||
@ -596,4 +596,139 @@ struct SettingCompressionMethod
|
||||
writeBinary(toString(), buf);
|
||||
}
|
||||
};
|
||||
|
||||
/// Настройка для выполнения распределённых подзапросов внутри секций IN или JOIN.
|
||||
enum class DistributedProductMode
|
||||
{
|
||||
DENY = 0, /// Запретить
|
||||
LOCAL, /// Конвертировать в локальный запрос
|
||||
GLOBAL, /// Конвертировать в глобальный запрос
|
||||
ALLOW /// Разрешить
|
||||
};
|
||||
|
||||
struct SettingDistributedProductMode
|
||||
{
|
||||
DistributedProductMode value;
|
||||
bool changed = false;
|
||||
|
||||
SettingDistributedProductMode(DistributedProductMode x) : value(x) {}
|
||||
|
||||
operator DistributedProductMode() const { return value; }
|
||||
SettingDistributedProductMode & operator= (DistributedProductMode x) { set(x); return *this; }
|
||||
|
||||
static DistributedProductMode getDistributedProductMode(const String & s)
|
||||
{
|
||||
if (s == "deny") return DistributedProductMode::DENY;
|
||||
if (s == "local") return DistributedProductMode::LOCAL;
|
||||
if (s == "global") return DistributedProductMode::GLOBAL;
|
||||
if (s == "allow") return DistributedProductMode::ALLOW;
|
||||
|
||||
throw Exception("Unknown distributed product mode: '" + s + "', must be one of 'deny', 'local', 'global', 'allow'",
|
||||
ErrorCodes::UNKNOWN_DISTRIBUTED_PRODUCT_MODE);
|
||||
}
|
||||
|
||||
String toString() const
|
||||
{
|
||||
const char * strings[] = {"deny", "local", "global", "allow"};
|
||||
if (value < DistributedProductMode::DENY || value > DistributedProductMode::ALLOW)
|
||||
throw Exception("Unknown distributed product mode", ErrorCodes::UNKNOWN_OVERFLOW_MODE);
|
||||
return strings[static_cast<size_t>(value)];
|
||||
}
|
||||
|
||||
void set(DistributedProductMode x)
|
||||
{
|
||||
value = x;
|
||||
changed = true;
|
||||
}
|
||||
|
||||
void set(const Field & x)
|
||||
{
|
||||
set(safeGet<const String &>(x));
|
||||
}
|
||||
|
||||
void set(const String & x)
|
||||
{
|
||||
set(getDistributedProductMode(x));
|
||||
}
|
||||
|
||||
void set(ReadBuffer & buf)
|
||||
{
|
||||
String x;
|
||||
readBinary(x, buf);
|
||||
set(x);
|
||||
}
|
||||
|
||||
void write(WriteBuffer & buf) const
|
||||
{
|
||||
writeBinary(toString(), buf);
|
||||
}
|
||||
};
|
||||
|
||||
/// Способ выполнения глобальных распределённых подзапросов.
|
||||
enum class GlobalSubqueriesMethod
|
||||
{
|
||||
PUSH = 0, /// Отправлять данные подзапроса на все удалённые серверы.
|
||||
PULL = 1, /// Удалённые серверы будут скачивать данные подзапроса с сервера-инициатора.
|
||||
};
|
||||
|
||||
struct SettingGlobalSubqueriesMethod
|
||||
{
|
||||
GlobalSubqueriesMethod value;
|
||||
bool changed = false;
|
||||
|
||||
SettingGlobalSubqueriesMethod(GlobalSubqueriesMethod x = GlobalSubqueriesMethod::PUSH) : value(x) {}
|
||||
|
||||
operator GlobalSubqueriesMethod() const { return value; }
|
||||
SettingGlobalSubqueriesMethod & operator= (GlobalSubqueriesMethod x) { set(x); return *this; }
|
||||
|
||||
static GlobalSubqueriesMethod getGlobalSubqueriesMethod(const String & s)
|
||||
{
|
||||
if (s == "push")
|
||||
return GlobalSubqueriesMethod::PUSH;
|
||||
if (s == "pull")
|
||||
return GlobalSubqueriesMethod::PULL;
|
||||
|
||||
throw Exception("Unknown global subqueries execution method: '" + s + "', must be one of 'push', 'pull'",
|
||||
ErrorCodes::UNKNOWN_GLOBAL_SUBQUERIES_METHOD);
|
||||
}
|
||||
|
||||
String toString() const
|
||||
{
|
||||
const char * strings[] = { "push", "pull" };
|
||||
|
||||
if (value < GlobalSubqueriesMethod::PUSH || value > GlobalSubqueriesMethod::PULL)
|
||||
throw Exception("Unknown global subqueries execution method", ErrorCodes::UNKNOWN_GLOBAL_SUBQUERIES_METHOD);
|
||||
|
||||
return strings[static_cast<size_t>(value)];
|
||||
}
|
||||
|
||||
void set(GlobalSubqueriesMethod x)
|
||||
{
|
||||
value = x;
|
||||
changed = true;
|
||||
}
|
||||
|
||||
void set(const Field & x)
|
||||
{
|
||||
set(safeGet<const String &>(x));
|
||||
}
|
||||
|
||||
void set(const String & x)
|
||||
{
|
||||
set(getGlobalSubqueriesMethod(x));
|
||||
}
|
||||
|
||||
void set(ReadBuffer & buf)
|
||||
{
|
||||
String x;
|
||||
readBinary(x, buf);
|
||||
set(x);
|
||||
}
|
||||
|
||||
void write(WriteBuffer & buf) const
|
||||
{
|
||||
writeBinary(toString(), buf);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
/// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке.
|
||||
void rewriteSelectExpressionList(const Names & column_names);
|
||||
|
||||
bool isUnionAllHead() const { return prev_union_all.isNull() && !next_union_all.isNull(); }
|
||||
bool isUnionAllHead() const { return (prev_union_all == nullptr) && !next_union_all.isNull(); }
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
@ -64,10 +64,16 @@ public:
|
||||
ASTPtr limit_offset;
|
||||
ASTPtr limit_length;
|
||||
ASTPtr settings;
|
||||
/// Предыдущий запрос SELECT в цепочке UNION ALL (не вставляется в children и не клонируется)
|
||||
ASTPtr prev_union_all;
|
||||
|
||||
/// Двусвязный список запросов SELECT внутри запроса UNION ALL.
|
||||
|
||||
/// Следующий запрос SELECT в цепочке UNION ALL, если такой есть
|
||||
ASTPtr next_union_all;
|
||||
/// Предыдущий запрос SELECT в цепочке UNION ALL (не вставляется в children и не клонируется)
|
||||
/// Указатель голый по следующим двум причинам:
|
||||
/// 1. чтобы предотвратить появление циклических зависимостей и, значит, утечки памяти;
|
||||
/// 2. библиотека Poco не поддерживает указателей наподобие std::weak_ptr.
|
||||
IAST * prev_union_all = nullptr;
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
@ -32,7 +32,35 @@ public:
|
||||
typedef std::vector<SharedPtr<IAST> > ASTs;
|
||||
ASTs children;
|
||||
StringRange range;
|
||||
bool is_visited = false;
|
||||
|
||||
/// Указатель на начало секции [NOT]IN или JOIN в которой включен этот узел,
|
||||
/// если имеется такая секция.
|
||||
IAST * enclosing_in_or_join = nullptr;
|
||||
|
||||
/// Атрибуты, которые нужны для некоторых алгоритмов на синтаксических деревьях.
|
||||
using Attributes = UInt32;
|
||||
Attributes attributes = 0;
|
||||
|
||||
/// Был ли узел посещён? (см. класс LogicalExpressionsOptimizer)
|
||||
static constexpr Attributes IsVisited = 1U;
|
||||
/// Был ли узел обработан? (см. класс InJoinSubqueriesPreprocessor)
|
||||
static constexpr Attributes IsPreprocessedForInJoinSubqueries = 1U << 1;
|
||||
/// Является ли узел секцией IN?
|
||||
static constexpr Attributes IsIn = 1U << 2;
|
||||
/// Является ли узел секцией NOT IN?
|
||||
static constexpr Attributes IsNotIn = 1U << 3;
|
||||
/// Является ли узел секцией JOIN?
|
||||
static constexpr Attributes IsJoin = 1U << 4;
|
||||
/// Имеет ли секция IN/NOT IN/JOIN атрибут GLOBAL?
|
||||
static constexpr Attributes IsGlobal = 1U << 5;
|
||||
|
||||
/** Глубина одного узла N - это глубина того запроса, которому принадлежит N.
|
||||
* Дальше глубина одного запроса определяется следующим образом:
|
||||
* - если запрос Q является главным, то select_query_depth(Q) = 0
|
||||
* - если запрос S является непосредственным подзапросом одного запроса R,
|
||||
* то select_query_depth(S) = select_query_depth(R) + 1
|
||||
*/
|
||||
UInt32 select_query_depth = 0;
|
||||
|
||||
/** Строка с полным запросом.
|
||||
* Этот указатель не дает ее удалить, пока range в нее ссылается.
|
||||
@ -64,11 +92,12 @@ public:
|
||||
/** Получить глубокую копию дерева. */
|
||||
virtual SharedPtr<IAST> clone() const = 0;
|
||||
|
||||
void clearVisited()
|
||||
/// Рекурсивно установить атрибуты в поддереве, корнем которого является текущий узел.
|
||||
void setAttributes(Attributes attributes_)
|
||||
{
|
||||
is_visited = false;
|
||||
for (ASTs::iterator it = children.begin(); it != children.end(); ++it)
|
||||
(*it)->is_visited = false;
|
||||
attributes |= attributes_;
|
||||
for (auto it : children)
|
||||
it->setAttributes(attributes_);
|
||||
}
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент и всё поддерево.
|
||||
|
@ -176,7 +176,8 @@ public:
|
||||
* Гарантируется, что структура таблицы не изменится за время жизни возвращенных потоков (то есть не будет ALTER, RENAME и DROP).
|
||||
*/
|
||||
virtual BlockOutputStreamPtr write(
|
||||
ASTPtr query)
|
||||
ASTPtr query,
|
||||
const Settings & settings)
|
||||
{
|
||||
throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
@ -308,7 +309,14 @@ private:
|
||||
mutable Poco::RWLock structure_lock;
|
||||
};
|
||||
|
||||
typedef std::vector<StoragePtr> StorageVector;
|
||||
typedef IStorage::TableStructureReadLocks TableLocks;
|
||||
using StorageVector = std::vector<StoragePtr>;
|
||||
using TableLocks = IStorage::TableStructureReadLocks;
|
||||
|
||||
/// имя таблицы -> таблица
|
||||
using Tables = std::map<String, StoragePtr>;
|
||||
|
||||
/// имя БД -> таблицы
|
||||
using Databases = std::map<String, Tables>;
|
||||
|
||||
|
||||
}
|
||||
|
24
dbms/include/DB/Storages/MergeTree/MarkRange.h
Normal file
24
dbms/include/DB/Storages/MergeTree/MarkRange.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstddef>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Пара засечек, определяющая диапазон строк в куске. Именно, диапазон имеет вид [begin * index_granularity, end * index_granularity).
|
||||
*/
|
||||
struct MarkRange
|
||||
{
|
||||
std::size_t begin;
|
||||
std::size_t end;
|
||||
|
||||
MarkRange() = default;
|
||||
MarkRange(const std::size_t begin, const std::size_t end) : begin{begin}, end{end} {}
|
||||
};
|
||||
|
||||
using MarkRanges = std::vector<MarkRange>;
|
||||
|
||||
|
||||
}
|
@ -30,12 +30,13 @@ public:
|
||||
ordered_names{column_names},
|
||||
min_bytes_to_use_direct_io(min_bytes_to_use_direct_io_), max_read_buffer_size(max_read_buffer_size_)
|
||||
{
|
||||
/** @note можно было бы просто поменять местами reverse в if и else ветках MergeTreeDataSelectExecutor,
|
||||
* а этот reverse убрать. */
|
||||
std::reverse(remaining_mark_ranges.begin(), remaining_mark_ranges.end());
|
||||
|
||||
/// inject columns required for defaults evaluation
|
||||
const auto injected_columns = injectRequiredColumns(column_names);
|
||||
/// insert injected columns into ordered columns list to avoid exception about different block structures
|
||||
ordered_names.insert(std::end(ordered_names), std::begin(injected_columns), std::end(injected_columns));
|
||||
should_reorder = !injected_columns.empty();
|
||||
|
||||
Names pre_column_names;
|
||||
|
||||
@ -43,11 +44,13 @@ public:
|
||||
{
|
||||
pre_column_names = prewhere_actions->getRequiredColumns();
|
||||
|
||||
injectRequiredColumns(pre_column_names);
|
||||
|
||||
if (pre_column_names.empty())
|
||||
pre_column_names.push_back(column_names[0]);
|
||||
|
||||
const auto injected_pre_columns = injectRequiredColumns(pre_column_names);
|
||||
if (!injected_pre_columns.empty())
|
||||
should_reorder = true;
|
||||
|
||||
const NameSet pre_name_set(pre_column_names.begin(), pre_column_names.end());
|
||||
/// Если выражение в PREWHERE - не столбец таблицы, не нужно отдавать наружу столбец с ним
|
||||
/// (от storage ожидают получить только столбцы таблицы).
|
||||
@ -135,13 +138,18 @@ protected:
|
||||
NameSet required_columns{std::begin(columns), std::end(columns)};
|
||||
NameSet injected_columns;
|
||||
|
||||
auto all_column_files_missing = true;
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
const auto & column_name = columns[i];
|
||||
|
||||
/// column has files and hence does not require evaluation
|
||||
if (owned_data_part->hasColumnFiles(column_name))
|
||||
{
|
||||
all_column_files_missing = false;
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto default_it = storage.column_defaults.find(column_name);
|
||||
/// columns has no explicit default expression
|
||||
@ -167,6 +175,14 @@ protected:
|
||||
}
|
||||
}
|
||||
|
||||
/// If all files are missing read at least one column to determine correct column sizes
|
||||
if (all_column_files_missing)
|
||||
{
|
||||
const auto minimum_size_column_name = owned_data_part->getMinimumSizeColumnName();
|
||||
columns.push_back(minimum_size_column_name);
|
||||
injected_columns.insert(minimum_size_column_name);
|
||||
}
|
||||
|
||||
return injected_columns;
|
||||
}
|
||||
|
||||
@ -216,7 +232,7 @@ protected:
|
||||
remaining_mark_ranges.pop_back();
|
||||
}
|
||||
progressImpl(Progress(res.rowsInFirstColumn(), res.bytes()));
|
||||
pre_reader->fillMissingColumns(res, ordered_names);
|
||||
pre_reader->fillMissingColumns(res, ordered_names, should_reorder);
|
||||
|
||||
/// Вычислим выражение в PREWHERE.
|
||||
prewhere_actions->execute(res);
|
||||
@ -382,6 +398,7 @@ private:
|
||||
|
||||
/// column names in specific order as expected by other stages
|
||||
Names ordered_names;
|
||||
bool should_reorder{false};
|
||||
|
||||
size_t min_bytes_to_use_direct_io;
|
||||
size_t max_read_buffer_size;
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
break;
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::SynchronousMergeOnInsert);
|
||||
storage.merge(0, true);
|
||||
storage.merge(0);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -177,6 +177,52 @@ public:
|
||||
|
||||
DataPart(MergeTreeData & storage_) : storage(storage_) {}
|
||||
|
||||
/// Returns the size of .bin file for column `name` if found, zero otherwise
|
||||
std::size_t getColumnSize(const String & name) const
|
||||
{
|
||||
if (checksums.empty())
|
||||
return {};
|
||||
|
||||
const auto & files = checksums.files;
|
||||
const auto bin_file_name = escapeForFileName(name) + ".bin";
|
||||
|
||||
/// Probably a logic error, not sure if this can ever happen if checksums are not empty
|
||||
if (0 == files.count(bin_file_name))
|
||||
return {};
|
||||
|
||||
return files.find(bin_file_name)->second.file_size;
|
||||
}
|
||||
|
||||
/** Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
|
||||
* If no checksums are present returns the name of the first physically existing column. */
|
||||
String getMinimumSizeColumnName() const
|
||||
{
|
||||
const auto & columns = storage.getColumnsList();
|
||||
const std::string * minimum_size_column = nullptr;
|
||||
auto minimum_size = std::numeric_limits<std::size_t>::max();
|
||||
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
if (!hasColumnFiles(column.name))
|
||||
continue;
|
||||
|
||||
const auto size = getColumnSize(column.name);
|
||||
if (size < minimum_size)
|
||||
{
|
||||
minimum_size = size;
|
||||
minimum_size_column = &column.name;
|
||||
}
|
||||
}
|
||||
|
||||
if (!minimum_size_column)
|
||||
throw Exception{
|
||||
"Could not find a column of minimum size in MergeTree",
|
||||
ErrorCodes::LOGICAL_ERROR
|
||||
};
|
||||
|
||||
return *minimum_size_column;
|
||||
}
|
||||
|
||||
MergeTreeData & storage;
|
||||
|
||||
size_t size = 0; /// в количестве засечек.
|
||||
@ -319,7 +365,7 @@ public:
|
||||
* Это делается только в случае to_detached, потому что считается, что в этом случае, точное имя не имеет значения.
|
||||
* Больше 10 попыток не делается, чтобы не оставалось слишком много мусорных директорий.
|
||||
*/
|
||||
while (try_no < 10 && Poco::File(dst_name()).exists())
|
||||
while (try_no < 10 && Poco::File(storage.full_path + dst_name()).exists())
|
||||
{
|
||||
LOG_WARNING(storage.log, "Directory " << dst_name() << " (to detach to) is already exist."
|
||||
" Will detach to directory with '_tryN' suffix.");
|
||||
@ -498,7 +544,7 @@ public:
|
||||
|
||||
|
||||
/// Некоторые операции над множеством кусков могут возвращать такой объект.
|
||||
/// Если не был вызван commit, деструктор откатывает операцию.
|
||||
/// Если не был вызван commit или rollback, деструктор откатывает операцию.
|
||||
class Transaction : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
@ -506,20 +552,25 @@ public:
|
||||
|
||||
void commit()
|
||||
{
|
||||
data = nullptr;
|
||||
removed_parts.clear();
|
||||
added_parts.clear();
|
||||
clear();
|
||||
}
|
||||
|
||||
void rollback()
|
||||
{
|
||||
if (data && (!parts_to_remove_on_rollback.empty() || !parts_to_add_on_rollback.empty()))
|
||||
{
|
||||
LOG_DEBUG(data->log, "Undoing transaction");
|
||||
data->replaceParts(parts_to_remove_on_rollback, parts_to_add_on_rollback, true);
|
||||
|
||||
clear();
|
||||
}
|
||||
}
|
||||
|
||||
~Transaction()
|
||||
{
|
||||
try
|
||||
{
|
||||
if (data && (!removed_parts.empty() || !added_parts.empty()))
|
||||
{
|
||||
LOG_DEBUG(data->log, "Undoing transaction");
|
||||
data->replaceParts(removed_parts, added_parts, true);
|
||||
}
|
||||
rollback();
|
||||
}
|
||||
catch(...)
|
||||
{
|
||||
@ -532,8 +583,15 @@ public:
|
||||
MergeTreeData * data = nullptr;
|
||||
|
||||
/// Что делать для отката операции.
|
||||
DataPartsVector removed_parts;
|
||||
DataPartsVector added_parts;
|
||||
DataPartsVector parts_to_remove_on_rollback;
|
||||
DataPartsVector parts_to_add_on_rollback;
|
||||
|
||||
void clear()
|
||||
{
|
||||
data = nullptr;
|
||||
parts_to_remove_on_rollback.clear();
|
||||
parts_to_add_on_rollback.clear();
|
||||
}
|
||||
};
|
||||
|
||||
/// Объект, помнящий какие временные файлы были созданы в директории с куском в ходе изменения (ALTER) его столбцов.
|
||||
@ -696,7 +754,7 @@ public:
|
||||
DataPartsVector renameTempPartAndReplace(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
|
||||
|
||||
/** Убирает из рабочего набора куски remove и добавляет куски add. add должны уже быть в all_data_parts.
|
||||
* Если clear_without_timeout, данные будут удалены при следующем clearOldParts, игнорируя old_parts_lifetime.
|
||||
* Если clear_without_timeout, данные будут удалены сразу, либо при следующем clearOldParts, игнорируя old_parts_lifetime.
|
||||
*/
|
||||
void replaceParts(const DataPartsVector & remove, const DataPartsVector & add, bool clear_without_timeout);
|
||||
|
||||
|
@ -54,8 +54,9 @@ public:
|
||||
/** Отменяет все мерджи. Все выполняющиеся сейчас вызовы mergeParts скоро бросят исключение.
|
||||
* Все новые вызовы будут бросать исключения, пока не будет вызван uncancelAll().
|
||||
*/
|
||||
bool cancelAll() { return canceled.exchange(true, std::memory_order_relaxed); }
|
||||
bool uncancelAll() { return canceled.exchange(false, std::memory_order_relaxed); }
|
||||
bool cancelAll() { return cancelled.exchange(true, std::memory_order_relaxed); }
|
||||
bool uncancelAll() { return cancelled.exchange(false, std::memory_order_relaxed); }
|
||||
bool isCancelled() const { return cancelled.load(std::memory_order_relaxed); }
|
||||
|
||||
private:
|
||||
MergeTreeData & data;
|
||||
@ -65,7 +66,7 @@ private:
|
||||
/// Когда в последний раз писали в лог, что место на диске кончилось (чтобы не писать об этом слишком часто).
|
||||
time_t disk_space_warning_time = 0;
|
||||
|
||||
std::atomic<bool> canceled{false};
|
||||
std::atomic<bool> cancelled{false};
|
||||
};
|
||||
|
||||
class MergeTreeMergeBlocker
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeReader.h>
|
||||
#include <DB/Storages/MergeTree/RangesInDataPart.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -33,22 +34,6 @@ private:
|
||||
|
||||
Logger * log;
|
||||
|
||||
struct RangesInDataPart
|
||||
{
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
size_t part_index_in_query;
|
||||
MarkRanges ranges;
|
||||
|
||||
RangesInDataPart() {}
|
||||
|
||||
RangesInDataPart(MergeTreeData::DataPartPtr data_part_, size_t part_index_in_query_)
|
||||
: data_part(data_part_), part_index_in_query(part_index_in_query_)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
typedef std::vector<RangesInDataPart> RangesInDataParts;
|
||||
|
||||
BlockInputStreams spreadMarkRangesAmongThreads(
|
||||
RangesInDataParts parts,
|
||||
size_t threads,
|
||||
|
422
dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h
Normal file
422
dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h
Normal file
@ -0,0 +1,422 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/NamesAndTypes.h>
|
||||
#include <DB/Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
#include <mutex>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// A batch of work for MergeTreeThreadBlockInputStream
|
||||
struct MergeTreeReadTask
|
||||
{
|
||||
/// data part which should be read while performing this task
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
/** Ranges to read from `data_part`.
|
||||
* Specified in reverse order for MergeTreeThreadBlockInputStream's convenience of calling .pop_back(). */
|
||||
MarkRanges mark_ranges;
|
||||
/// for virtual `part_index` virtual column
|
||||
std::size_t part_index_in_query;
|
||||
/// ordered list of column names used in this query, allows returning blocks with consistent ordering
|
||||
const Names & ordered_names;
|
||||
/// used to determine whether column should be filtered during PREWHERE or WHERE
|
||||
const NameSet & column_name_set;
|
||||
/// column names to read during WHERE
|
||||
const NamesAndTypesList & columns;
|
||||
/// column names to read during PREWHERE
|
||||
const NamesAndTypesList & pre_columns;
|
||||
/// should PREWHERE column be returned to requesting side?
|
||||
const bool remove_prewhere_column;
|
||||
/// resulting block may require reordering in accordance with `ordered_names`
|
||||
const bool should_reorder;
|
||||
|
||||
MergeTreeReadTask(
|
||||
const MergeTreeData::DataPartPtr & data_part, const MarkRanges & ranges, const std::size_t part_index_in_query,
|
||||
const Names & ordered_names, const NameSet & column_name_set, const NamesAndTypesList & columns,
|
||||
const NamesAndTypesList & pre_columns, const bool remove_prewhere_column, const bool should_reorder)
|
||||
: data_part{data_part}, mark_ranges{ranges}, part_index_in_query{part_index_in_query},
|
||||
ordered_names{ordered_names}, column_name_set{column_name_set}, columns{columns}, pre_columns{pre_columns},
|
||||
remove_prewhere_column{remove_prewhere_column}, should_reorder{should_reorder}
|
||||
{}
|
||||
};
|
||||
|
||||
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
|
||||
|
||||
/** Provides read tasks for MergeTreeThreadBlockInputStream`s in fine-grained batches, allowing for more
|
||||
* uniform distribution of work amongst multiple threads. All parts and their ranges are divided into `threads`
|
||||
* workloads with at most `sum_marks / threads` marks. Then, threads are performing reads from these workloads
|
||||
* in "sequential" manner, requesting work in small batches. As soon as some thread some thread has exhausted
|
||||
* it's workload, it either is signaled that no more work is available (`do_not_steal_tasks == false`) or
|
||||
* continues taking small batches from other threads' workloads (`do_not_steal_tasks == true`).
|
||||
*/
|
||||
class MergeTreeReadPool
|
||||
{
|
||||
public:
|
||||
MergeTreeReadPool(
|
||||
const std::size_t threads, const std::size_t sum_marks, const std::size_t min_marks_for_concurrent_read,
|
||||
RangesInDataParts parts, MergeTreeData & data, const ExpressionActionsPtr & prewhere_actions,
|
||||
const String & prewhere_column_name, const bool check_columns, const Names & column_names,
|
||||
const bool do_not_steal_tasks = false)
|
||||
: data{data}, column_names{column_names}, do_not_steal_tasks{do_not_steal_tasks}
|
||||
{
|
||||
const auto per_part_sum_marks = fillPerPartInfo(parts, prewhere_actions, prewhere_column_name, check_columns);
|
||||
fillPerThreadInfo(threads, sum_marks, per_part_sum_marks, parts, min_marks_for_concurrent_read);
|
||||
}
|
||||
|
||||
MergeTreeReadPool(const MergeTreeReadPool &) = delete;
|
||||
MergeTreeReadPool & operator=(const MergeTreeReadPool &) = delete;
|
||||
|
||||
MergeTreeReadTaskPtr getTask(const std::size_t min_marks_to_read, const std::size_t thread)
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{mutex};
|
||||
|
||||
if (remaining_thread_tasks.empty())
|
||||
return nullptr;
|
||||
|
||||
const auto tasks_remaining_for_this_thread = !threads_tasks[thread].sum_marks_in_parts.empty();
|
||||
if (!tasks_remaining_for_this_thread && do_not_steal_tasks)
|
||||
return nullptr;
|
||||
|
||||
const auto thread_idx = tasks_remaining_for_this_thread ? thread : *std::begin(remaining_thread_tasks);
|
||||
auto & thread_tasks = threads_tasks[thread_idx];
|
||||
|
||||
auto & thread_task = thread_tasks.parts_and_ranges.back();
|
||||
const auto part_idx = thread_task.part_idx;
|
||||
|
||||
auto & part = parts[part_idx];
|
||||
auto & marks_in_part = thread_tasks.sum_marks_in_parts.back();
|
||||
|
||||
/// Берём весь кусок, если он достаточно мал
|
||||
auto need_marks = std::min(marks_in_part, min_marks_to_read);
|
||||
|
||||
/// Не будем оставлять в куске слишком мало строк.
|
||||
if (marks_in_part > need_marks &&
|
||||
marks_in_part - need_marks < min_marks_to_read)
|
||||
need_marks = marks_in_part;
|
||||
|
||||
MarkRanges ranges_to_get_from_part;
|
||||
|
||||
/// Возьмем весь кусок, если он достаточно мал.
|
||||
if (marks_in_part <= need_marks)
|
||||
{
|
||||
const auto marks_to_get_from_range = marks_in_part;
|
||||
|
||||
/** Отрезки уже перечислены справа налево, reverse изначально сделан в MergeTreeDataSelectExecutor и
|
||||
* поддержан в fillPerThreadInfo. */
|
||||
ranges_to_get_from_part = thread_task.ranges;
|
||||
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
|
||||
thread_tasks.parts_and_ranges.pop_back();
|
||||
thread_tasks.sum_marks_in_parts.pop_back();
|
||||
|
||||
if (thread_tasks.sum_marks_in_parts.empty())
|
||||
remaining_thread_tasks.erase(thread_idx);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Цикл по отрезкам куска.
|
||||
while (need_marks > 0 && !thread_task.ranges.empty())
|
||||
{
|
||||
auto & range = thread_task.ranges.back();
|
||||
|
||||
const std::size_t marks_in_range = range.end - range.begin;
|
||||
const std::size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
if (range.begin == range.end)
|
||||
{
|
||||
std::swap(range, thread_task.ranges.back());
|
||||
thread_task.ranges.pop_back();
|
||||
}
|
||||
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
need_marks -= marks_to_get_from_range;
|
||||
}
|
||||
|
||||
/** Перечислим справа налево, чтобы MergeTreeThreadBlockInputStream забирал
|
||||
* отрезки с помощью .pop_back() (их порядок был сменен на "слева направо"
|
||||
* из-за .pop_back() в этой ветке). */
|
||||
std::reverse(std::begin(ranges_to_get_from_part), std::end(ranges_to_get_from_part));
|
||||
}
|
||||
|
||||
return std::make_unique<MergeTreeReadTask>(
|
||||
part.data_part, ranges_to_get_from_part, part.part_index_in_query, column_names,
|
||||
per_part_column_name_set[part_idx], per_part_columns[part_idx], per_part_pre_columns[part_idx],
|
||||
per_part_remove_prewhere_column[part_idx], per_part_should_reorder[part_idx]);
|
||||
}
|
||||
|
||||
public:
|
||||
std::vector<std::size_t> fillPerPartInfo(
|
||||
RangesInDataParts & parts, const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column_name,
|
||||
const bool check_columns)
|
||||
{
|
||||
std::vector<std::size_t> per_part_sum_marks;
|
||||
|
||||
for (const auto i : ext::range(0, parts.size()))
|
||||
{
|
||||
auto & part = parts[i];
|
||||
|
||||
/// Посчитаем засечки для каждого куска.
|
||||
size_t sum_marks = 0;
|
||||
/// Отрезки уже перечислены справа налево, reverse в MergeTreeDataSelectExecutor.
|
||||
for (const auto & range : part.ranges)
|
||||
sum_marks += range.end - range.begin;
|
||||
|
||||
per_part_sum_marks.push_back(sum_marks);
|
||||
|
||||
per_part_columns_lock.push_back(std::make_unique<Poco::ScopedReadRWLock>(
|
||||
part.data_part->columns_lock));
|
||||
|
||||
/// inject column names required for DEFAULT evaluation in current part
|
||||
auto required_column_names = column_names;
|
||||
|
||||
const auto injected_columns = injectRequiredColumns(part.data_part, required_column_names);
|
||||
auto should_reoder = !injected_columns.empty();
|
||||
|
||||
Names required_pre_column_names;
|
||||
|
||||
if (prewhere_actions)
|
||||
{
|
||||
/// collect columns required for PREWHERE evaluation
|
||||
required_pre_column_names = prewhere_actions->getRequiredColumns();
|
||||
|
||||
/// there must be at least one column required for PREWHERE
|
||||
if (required_pre_column_names.empty())
|
||||
required_pre_column_names.push_back(required_column_names[0]);
|
||||
|
||||
/// PREWHERE columns may require some additional columns for DEFAULT evaluation
|
||||
const auto injected_pre_columns = injectRequiredColumns(part.data_part, required_pre_column_names);
|
||||
if (!injected_pre_columns.empty())
|
||||
should_reoder = true;
|
||||
|
||||
/// will be used to distinguish between PREWHERE and WHERE columns when applying filter
|
||||
const NameSet pre_name_set{
|
||||
std::begin(required_pre_column_names), std::end(required_pre_column_names)
|
||||
};
|
||||
/** Если выражение в PREWHERE - не столбец таблицы, не нужно отдавать наружу столбец с ним
|
||||
* (от storage ожидают получить только столбцы таблицы). */
|
||||
per_part_remove_prewhere_column.push_back(0 == pre_name_set.count(prewhere_column_name));
|
||||
|
||||
Names post_column_names;
|
||||
for (const auto & name : required_column_names)
|
||||
if (!pre_name_set.count(name))
|
||||
post_column_names.push_back(name);
|
||||
|
||||
required_column_names = post_column_names;
|
||||
}
|
||||
else
|
||||
per_part_remove_prewhere_column.push_back(false);
|
||||
|
||||
per_part_column_name_set.emplace_back(std::begin(required_column_names), std::end(required_column_names));
|
||||
|
||||
if (check_columns)
|
||||
{
|
||||
/** Под part->columns_lock проверим, что все запрошенные столбцы в куске того же типа, что в таблице.
|
||||
* Это может быть не так во время ALTER MODIFY. */
|
||||
if (!required_pre_column_names.empty())
|
||||
data.check(part.data_part->columns, required_pre_column_names);
|
||||
if (!required_column_names.empty())
|
||||
data.check(part.data_part->columns, required_column_names);
|
||||
|
||||
per_part_pre_columns.push_back(data.getColumnsList().addTypes(required_pre_column_names));
|
||||
per_part_columns.push_back(data.getColumnsList().addTypes(required_column_names));
|
||||
}
|
||||
else
|
||||
{
|
||||
per_part_pre_columns.push_back(part.data_part->columns.addTypes(required_pre_column_names));
|
||||
per_part_columns.push_back(part.data_part->columns.addTypes(required_column_names));
|
||||
}
|
||||
|
||||
per_part_should_reorder.push_back(should_reoder);
|
||||
|
||||
this->parts.push_back({ part.data_part, part.part_index_in_query });
|
||||
}
|
||||
|
||||
return per_part_sum_marks;
|
||||
}
|
||||
|
||||
void fillPerThreadInfo(
|
||||
const std::size_t threads, const std::size_t sum_marks, std::vector<std::size_t> per_part_sum_marks,
|
||||
RangesInDataParts & parts, const std::size_t min_marks_for_concurrent_read)
|
||||
{
|
||||
threads_tasks.resize(threads);
|
||||
|
||||
const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1;
|
||||
|
||||
for (std::size_t i = 0; i < threads && !parts.empty(); ++i)
|
||||
{
|
||||
auto need_marks = min_marks_per_thread;
|
||||
|
||||
while (need_marks > 0 && !parts.empty())
|
||||
{
|
||||
const auto part_idx = parts.size() - 1;
|
||||
RangesInDataPart & part = parts.back();
|
||||
size_t & marks_in_part = per_part_sum_marks.back();
|
||||
|
||||
/// Не будем брать из куска слишком мало строк.
|
||||
if (marks_in_part >= min_marks_for_concurrent_read &&
|
||||
need_marks < min_marks_for_concurrent_read)
|
||||
need_marks = min_marks_for_concurrent_read;
|
||||
|
||||
/// Не будем оставлять в куске слишком мало строк.
|
||||
if (marks_in_part > need_marks &&
|
||||
marks_in_part - need_marks < min_marks_for_concurrent_read)
|
||||
need_marks = marks_in_part;
|
||||
|
||||
MarkRanges ranges_to_get_from_part;
|
||||
size_t marks_in_ranges = need_marks;
|
||||
|
||||
/// Возьмем весь кусок, если он достаточно мал.
|
||||
if (marks_in_part <= need_marks)
|
||||
{
|
||||
/// Оставим отрезки перечисленными справа налево для удобства использования .pop_back() в .getTask()
|
||||
ranges_to_get_from_part = part.ranges;
|
||||
marks_in_ranges = marks_in_part;
|
||||
|
||||
need_marks -= marks_in_part;
|
||||
parts.pop_back();
|
||||
per_part_sum_marks.pop_back();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Цикл по отрезкам куска.
|
||||
while (need_marks > 0)
|
||||
{
|
||||
if (part.ranges.empty())
|
||||
throw Exception("Unexpected end of ranges while spreading marks among threads", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
MarkRange & range = part.ranges.back();
|
||||
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
need_marks -= marks_to_get_from_range;
|
||||
if (range.begin == range.end)
|
||||
part.ranges.pop_back();
|
||||
}
|
||||
|
||||
/** Вновь перечислим отрезки справа налево, чтобы .getTask() мог забирать их
|
||||
* с помощью .pop_back() (их порядок был сменен на "слева направо"
|
||||
* из-за .pop_back() в этой ветке). */
|
||||
std::reverse(std::begin(ranges_to_get_from_part), std::end(ranges_to_get_from_part));
|
||||
}
|
||||
|
||||
threads_tasks[i].parts_and_ranges.push_back({ part_idx, ranges_to_get_from_part });
|
||||
threads_tasks[i].sum_marks_in_parts.push_back(marks_in_ranges);
|
||||
if (marks_in_ranges != 0)
|
||||
remaining_thread_tasks.insert(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Если некоторых запрошенных столбцов нет в куске,
|
||||
* то выясняем, какие столбцы может быть необходимо дополнительно прочитать,
|
||||
* чтобы можно было вычислить DEFAULT выражение для этих столбцов.
|
||||
* Добавляет их в columns. */
|
||||
NameSet injectRequiredColumns(const MergeTreeData::DataPartPtr & part, Names & columns) const
|
||||
{
|
||||
NameSet required_columns{std::begin(columns), std::end(columns)};
|
||||
NameSet injected_columns;
|
||||
|
||||
auto all_column_files_missing = true;
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
const auto & column_name = columns[i];
|
||||
|
||||
/// column has files and hence does not require evaluation
|
||||
if (part->hasColumnFiles(column_name))
|
||||
{
|
||||
all_column_files_missing = false;
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto default_it = data.column_defaults.find(column_name);
|
||||
/// columns has no explicit default expression
|
||||
if (default_it == std::end(data.column_defaults))
|
||||
continue;
|
||||
|
||||
/// collect identifiers required for evaluation
|
||||
IdentifierNameSet identifiers;
|
||||
default_it->second.expression->collectIdentifierNames(identifiers);
|
||||
|
||||
for (const auto & identifier : identifiers)
|
||||
{
|
||||
if (data.hasColumn(identifier))
|
||||
{
|
||||
/// ensure each column is added only once
|
||||
if (required_columns.count(identifier) == 0)
|
||||
{
|
||||
columns.emplace_back(identifier);
|
||||
required_columns.emplace(identifier);
|
||||
injected_columns.emplace(identifier);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Добавить столбец минимального размера.
|
||||
* Используется в случае, когда ни один столбец не нужен или файлы отсутствуют, но нужно хотя бы знать количество строк.
|
||||
* Добавляет в columns.
|
||||
*/
|
||||
if (all_column_files_missing)
|
||||
{
|
||||
const auto minimum_size_column_name = part->getMinimumSizeColumnName();
|
||||
columns.push_back(minimum_size_column_name);
|
||||
/// correctly report added column
|
||||
injected_columns.insert(columns.back());
|
||||
}
|
||||
|
||||
return injected_columns;
|
||||
}
|
||||
|
||||
std::vector<std::unique_ptr<Poco::ScopedReadRWLock>> per_part_columns_lock;
|
||||
MergeTreeData & data;
|
||||
Names column_names;
|
||||
const bool do_not_steal_tasks;
|
||||
std::vector<NameSet> per_part_column_name_set;
|
||||
std::vector<NamesAndTypesList> per_part_columns;
|
||||
std::vector<NamesAndTypesList> per_part_pre_columns;
|
||||
/// @todo actually all of these values are either true or false for the whole query, thus no vector required
|
||||
std::vector<char> per_part_remove_prewhere_column;
|
||||
std::vector<char> per_part_should_reorder;
|
||||
|
||||
struct Part
|
||||
{
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
std::size_t part_index_in_query;
|
||||
};
|
||||
|
||||
std::vector<Part> parts;
|
||||
|
||||
struct ThreadTask
|
||||
{
|
||||
struct PartIndexAndRange
|
||||
{
|
||||
std::size_t part_idx;
|
||||
MarkRanges ranges;
|
||||
};
|
||||
|
||||
std::vector<PartIndexAndRange> parts_and_ranges;
|
||||
std::vector<std::size_t> sum_marks_in_parts;
|
||||
};
|
||||
|
||||
std::vector<ThreadTask> threads_tasks;
|
||||
|
||||
std::unordered_set<std::size_t> remaining_thread_tasks;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
};
|
||||
|
||||
using MergeTreeReadPoolPtr = std::shared_ptr<MergeTreeReadPool>;
|
||||
|
||||
|
||||
}
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Storages/MarkCache.h>
|
||||
#include <DB/Storages/MergeTree/MarkRange.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
#include <DB/DataTypes/DataTypeNested.h>
|
||||
@ -17,37 +18,24 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Пара засечек, определяющая диапазон строк в куске. Именно, диапазон имеет вид [begin * index_granularity, end * index_granularity).
|
||||
*/
|
||||
struct MarkRange
|
||||
{
|
||||
size_t begin;
|
||||
size_t end;
|
||||
|
||||
MarkRange() {}
|
||||
MarkRange(size_t begin_, size_t end_) : begin(begin_), end(end_) {}
|
||||
};
|
||||
|
||||
typedef std::vector<MarkRange> MarkRanges;
|
||||
|
||||
|
||||
/** Умеет читать данные между парой засечек из одного куска. При чтении последовательных отрезков не делает лишних seek-ов.
|
||||
* При чтении почти последовательных отрезков делает seek-и быстро, не выбрасывая содержимое буфера.
|
||||
*/
|
||||
class MergeTreeReader
|
||||
{
|
||||
typedef std::map<std::string, ColumnPtr> OffsetColumns;
|
||||
using OffsetColumns = std::map<std::string, ColumnPtr>;
|
||||
using ValueSizeMap = std::map<std::string, double>;
|
||||
|
||||
public:
|
||||
MergeTreeReader(const String & path_, /// Путь к куску
|
||||
const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_,
|
||||
UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_,
|
||||
MergeTreeData & storage_, const MarkRanges & all_mark_ranges,
|
||||
size_t aio_threshold_, size_t max_read_buffer_size_)
|
||||
: path(path_), data_part(data_part), part_name(data_part->name), columns(columns_),
|
||||
uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_),
|
||||
storage(storage_), all_mark_ranges(all_mark_ranges),
|
||||
aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_)
|
||||
MergeTreeReader(const String & path, /// Путь к куску
|
||||
const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns,
|
||||
UncompressedCache * uncompressed_cache, MarkCache * mark_cache,
|
||||
MergeTreeData & storage, const MarkRanges & all_mark_ranges,
|
||||
size_t aio_threshold, size_t max_read_buffer_size, const ValueSizeMap & avg_value_size_hints = ValueSizeMap{})
|
||||
: avg_value_size_hints(avg_value_size_hints), path(path), data_part(data_part), columns(columns),
|
||||
uncompressed_cache(uncompressed_cache), mark_cache(mark_cache), storage(storage),
|
||||
all_mark_ranges(all_mark_ranges), aio_threshold(aio_threshold), max_read_buffer_size(max_read_buffer_size)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -59,11 +47,13 @@ public:
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
storage.reportBrokenPart(part_name);
|
||||
storage.reportBrokenPart(data_part->name);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
const ValueSizeMap & getAvgValueSizeHints() const { return avg_value_size_hints; }
|
||||
|
||||
/** Если столбцов нет в блоке, добавляет их, если есть - добавляет прочитанные значения к ним в конец.
|
||||
* Не добавляет столбцы, для которых нет файлов. Чтобы их добавить, нужно вызвать fillMissingColumns.
|
||||
* В блоке должно быть либо ни одного столбца из columns, либо все, для которых есть файлы.
|
||||
@ -74,20 +64,14 @@ public:
|
||||
{
|
||||
size_t max_rows_to_read = (to_mark - from_mark) * storage.index_granularity;
|
||||
|
||||
/** Для некоторых столбцов файлы с данными могут отсутствовать.
|
||||
* Это бывает для старых кусков, после добавления новых столбцов в структуру таблицы.
|
||||
*/
|
||||
auto has_missing_columns = false;
|
||||
|
||||
/// Указатели на столбцы смещений, общие для столбцов из вложенных структур данных
|
||||
/// Если append, все значения nullptr, и offset_columns используется только для проверки, что столбец смещений уже прочитан.
|
||||
OffsetColumns offset_columns;
|
||||
const auto read_column = [&] (const NameAndTypePair & it) {
|
||||
|
||||
for (const NameAndTypePair & it : columns)
|
||||
{
|
||||
if (streams.end() == streams.find(it.name))
|
||||
{
|
||||
has_missing_columns = true;
|
||||
return;
|
||||
}
|
||||
continue;
|
||||
|
||||
/// Все столбцы уже есть в блоке. Будем добавлять значения в конец.
|
||||
bool append = res.has(it.name);
|
||||
@ -120,24 +104,12 @@ public:
|
||||
|
||||
if (!append && column.column->size())
|
||||
res.insert(column);
|
||||
};
|
||||
|
||||
for (const NameAndTypePair & it : columns)
|
||||
read_column(it);
|
||||
|
||||
if (has_missing_columns && !res)
|
||||
{
|
||||
addMinimumSizeColumn();
|
||||
/// minimum size column is necessarily at list's front
|
||||
read_column(columns.front());
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
|
||||
{
|
||||
storage.reportBrokenPart(part_name);
|
||||
}
|
||||
storage.reportBrokenPart(data_part->name);
|
||||
|
||||
/// Более хорошая диагностика.
|
||||
throw Exception(e.message() + "\n(while reading from part " + path + " from mark " + toString(from_mark) + " to "
|
||||
@ -145,66 +117,19 @@ public:
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
storage.reportBrokenPart(part_name);
|
||||
storage.reportBrokenPart(data_part->name);
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Добавить столбец минимального размера.
|
||||
* Используется в случае, когда ни один столбец не нужен, но нужно хотя бы знать количество строк.
|
||||
* Добавляет в columns.
|
||||
*/
|
||||
void addMinimumSizeColumn()
|
||||
{
|
||||
const auto get_column_size = [this] (const String & name) {
|
||||
const auto & files = data_part->checksums.files;
|
||||
|
||||
const auto escaped_name = escapeForFileName(name);
|
||||
const auto bin_file_name = escaped_name + ".bin";
|
||||
const auto mrk_file_name = escaped_name + ".mrk";
|
||||
|
||||
return files.find(bin_file_name)->second.file_size + files.find(mrk_file_name)->second.file_size;
|
||||
};
|
||||
|
||||
const auto & storage_columns = storage.getColumnsList();
|
||||
const NameAndTypePair * minimum_size_column = nullptr;
|
||||
auto minimum_size = std::numeric_limits<size_t>::max();
|
||||
|
||||
for (const auto & column : storage_columns)
|
||||
{
|
||||
if (!data_part->hasColumnFiles(column.name))
|
||||
continue;
|
||||
|
||||
const auto size = get_column_size(column.name);
|
||||
if (size < minimum_size)
|
||||
{
|
||||
minimum_size = size;
|
||||
minimum_size_column = &column;
|
||||
}
|
||||
}
|
||||
|
||||
if (!minimum_size_column)
|
||||
throw Exception{
|
||||
"could not find a column of minimum size in MergeTree",
|
||||
ErrorCodes::LOGICAL_ERROR
|
||||
};
|
||||
|
||||
addStream(minimum_size_column->name, *minimum_size_column->type, all_mark_ranges);
|
||||
columns.emplace(std::begin(columns), *minimum_size_column);
|
||||
|
||||
added_minimum_size_column = &columns.front();
|
||||
}
|
||||
|
||||
|
||||
/** Добавляет в блок недостающие столбцы из ordered_names, состоящие из значений по-умолчанию.
|
||||
* Недостающие столбцы добавляются в позиции, такие же как в ordered_names.
|
||||
* Если был добавлен хотя бы один столбец - то все столбцы в блоке переупорядочиваются как в ordered_names.
|
||||
*/
|
||||
void fillMissingColumns(Block & res, const Names & ordered_names)
|
||||
void fillMissingColumns(Block & res, const Names & ordered_names, const bool always_reorder = false)
|
||||
{
|
||||
fillMissingColumnsImpl(res, ordered_names, false);
|
||||
fillMissingColumnsImpl(res, ordered_names, always_reorder);
|
||||
}
|
||||
|
||||
/** То же самое, но всегда переупорядочивает столбцы в блоке, как в ordered_names
|
||||
@ -220,16 +145,14 @@ private:
|
||||
{
|
||||
MarkCache::MappedPtr marks;
|
||||
ReadBuffer * data_buffer;
|
||||
Poco::SharedPtr<CachedCompressedReadBuffer> cached_buffer;
|
||||
Poco::SharedPtr<CompressedReadBufferFromFile> non_cached_buffer;
|
||||
std::unique_ptr<CachedCompressedReadBuffer> cached_buffer;
|
||||
std::unique_ptr<CompressedReadBufferFromFile> non_cached_buffer;
|
||||
std::string path_prefix;
|
||||
size_t max_mark_range;
|
||||
|
||||
/// Используется в качестве подсказки, чтобы уменьшить количество реаллокаций при создании столбца переменной длины.
|
||||
double avg_value_size_hint = 0;
|
||||
|
||||
Stream(const String & path_prefix_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const MarkRanges & all_mark_ranges,
|
||||
size_t aio_threshold, size_t max_read_buffer_size)
|
||||
Stream(
|
||||
const String & path_prefix_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache,
|
||||
const MarkRanges & all_mark_ranges, size_t aio_threshold, size_t max_read_buffer_size)
|
||||
: path_prefix(path_prefix_)
|
||||
{
|
||||
loadMarks(mark_cache);
|
||||
@ -281,15 +204,15 @@ private:
|
||||
|
||||
if (uncompressed_cache)
|
||||
{
|
||||
cached_buffer = new CachedCompressedReadBuffer(path_prefix + ".bin", uncompressed_cache,
|
||||
estimated_size, aio_threshold, buffer_size);
|
||||
data_buffer = &*cached_buffer;
|
||||
cached_buffer = std::make_unique<CachedCompressedReadBuffer>(
|
||||
path_prefix + ".bin", uncompressed_cache, estimated_size, aio_threshold, buffer_size);
|
||||
data_buffer = cached_buffer.get();
|
||||
}
|
||||
else
|
||||
{
|
||||
non_cached_buffer = new CompressedReadBufferFromFile(path_prefix + ".bin", estimated_size,
|
||||
aio_threshold, buffer_size);
|
||||
data_buffer = &*non_cached_buffer;
|
||||
non_cached_buffer = std::make_unique<CompressedReadBufferFromFile>(
|
||||
path_prefix + ".bin", estimated_size, aio_threshold, buffer_size);
|
||||
data_buffer = non_cached_buffer.get();
|
||||
}
|
||||
}
|
||||
|
||||
@ -350,20 +273,20 @@ private:
|
||||
|
||||
typedef std::map<std::string, std::unique_ptr<Stream> > FileStreams;
|
||||
|
||||
/// Используется в качестве подсказки, чтобы уменьшить количество реаллокаций при создании столбца переменной длины.
|
||||
ValueSizeMap avg_value_size_hints;
|
||||
String path;
|
||||
const MergeTreeData::DataPartPtr & data_part;
|
||||
String part_name;
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
FileStreams streams;
|
||||
|
||||
/// Запрашиваемые столбцы. Возможно, с добавлением minimum_size_column.
|
||||
/// Запрашиваемые столбцы.
|
||||
NamesAndTypesList columns;
|
||||
const NameAndTypePair * added_minimum_size_column = nullptr;
|
||||
|
||||
UncompressedCache * uncompressed_cache;
|
||||
MarkCache * mark_cache;
|
||||
|
||||
MergeTreeData & storage;
|
||||
const MarkRanges & all_mark_ranges;
|
||||
MarkRanges all_mark_ranges;
|
||||
size_t aio_threshold;
|
||||
size_t max_read_buffer_size;
|
||||
|
||||
@ -386,14 +309,16 @@ private:
|
||||
+ ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
|
||||
|
||||
if (!streams.count(size_name))
|
||||
streams.emplace(size_name, std::unique_ptr<Stream>(new Stream(
|
||||
path + escaped_size_name, uncompressed_cache, mark_cache, all_mark_ranges, aio_threshold, max_read_buffer_size)));
|
||||
streams.emplace(size_name, std::make_unique<Stream>(
|
||||
path + escaped_size_name, uncompressed_cache, mark_cache,
|
||||
all_mark_ranges, aio_threshold, max_read_buffer_size));
|
||||
|
||||
addStream(name, *type_arr->getNestedType(), all_mark_ranges, level + 1);
|
||||
}
|
||||
else
|
||||
streams[name].reset(new Stream(
|
||||
path + escaped_column_name, uncompressed_cache, mark_cache, all_mark_ranges, aio_threshold, max_read_buffer_size));
|
||||
streams.emplace(name, std::make_unique<Stream>(
|
||||
path + escaped_column_name, uncompressed_cache, mark_cache,
|
||||
all_mark_ranges, aio_threshold, max_read_buffer_size));
|
||||
}
|
||||
|
||||
|
||||
@ -453,8 +378,9 @@ private:
|
||||
else
|
||||
{
|
||||
Stream & stream = *streams[name];
|
||||
double & avg_value_size_hint = avg_value_size_hints[name];
|
||||
stream.seekToMark(from_mark);
|
||||
type.deserializeBinary(column, *stream.data_buffer, max_rows_to_read, stream.avg_value_size_hint);
|
||||
type.deserializeBinary(column, *stream.data_buffer, max_rows_to_read, avg_value_size_hint);
|
||||
|
||||
/// Вычисление подсказки о среднем размере значения.
|
||||
size_t column_size = column.size();
|
||||
@ -463,10 +389,10 @@ private:
|
||||
double current_avg_value_size = static_cast<double>(column.byteSize()) / column_size;
|
||||
|
||||
/// Эвристика, чтобы при изменениях, значение avg_value_size_hint быстро росло, но медленно уменьшалось.
|
||||
if (current_avg_value_size > stream.avg_value_size_hint)
|
||||
stream.avg_value_size_hint = current_avg_value_size;
|
||||
else if (current_avg_value_size * 2 < stream.avg_value_size_hint)
|
||||
stream.avg_value_size_hint = (current_avg_value_size + stream.avg_value_size_hint * 3) / 4;
|
||||
if (current_avg_value_size > avg_value_size_hint)
|
||||
avg_value_size_hint = current_avg_value_size;
|
||||
else if (current_avg_value_size * 2 < avg_value_size_hint)
|
||||
avg_value_size_hint = (current_avg_value_size + avg_value_size_hint * 3) / 4;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -548,15 +474,6 @@ private:
|
||||
if (should_evaluate_defaults)
|
||||
evaluateMissingDefaults(res, columns, storage.column_defaults, storage.context);
|
||||
|
||||
/// remove added column to ensure same content among all blocks
|
||||
if (added_minimum_size_column)
|
||||
{
|
||||
res.erase(0);
|
||||
streams.erase(added_minimum_size_column->name);
|
||||
columns.erase(std::begin(columns));
|
||||
added_minimum_size_column = nullptr;
|
||||
}
|
||||
|
||||
/// sort columns to ensure consistent order among all blocks
|
||||
if (should_sort)
|
||||
{
|
||||
@ -566,12 +483,6 @@ private:
|
||||
if (res.has(name))
|
||||
ordered_block.insert(res.getByName(name));
|
||||
|
||||
if (res.columns() != ordered_block.columns())
|
||||
throw Exception{
|
||||
"Ordered block has different number of columns than original one:\n" +
|
||||
ordered_block.dumpNames() + "\nvs.\n" + res.dumpNames(),
|
||||
ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
std::swap(res, ordered_block);
|
||||
}
|
||||
}
|
||||
|
@ -66,7 +66,7 @@ struct MergeTreeSettings
|
||||
size_t replicated_logs_to_keep = 100;
|
||||
|
||||
/// Настройки минимального количества битых данных, при котором отказываться автоматически их удалять.
|
||||
size_t max_suspicious_broken_parts = 5;
|
||||
size_t max_suspicious_broken_parts = 10;
|
||||
|
||||
/// Максимальное количество ошибок при загрузке кусков, при котором ReplicatedMergeTree соглашается запускаться.
|
||||
size_t replicated_max_unexpected_parts = 3;
|
||||
|
@ -0,0 +1,354 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DB/Storages/MergeTree/PKCondition.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeReader.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeReadPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Used in conjunction with MergeTreeReadPool, asking it for more work to do and performing whatever reads it is asked
|
||||
* to perform. */
|
||||
class MergeTreeThreadBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
/// "thread" index (there are N threads and each thread is assigned index in interval [0..N-1])
|
||||
std::size_t thread;
|
||||
public:
|
||||
MergeTreeThreadBlockInputStream(
|
||||
const std::size_t thread,
|
||||
const MergeTreeReadPoolPtr & pool, const std::size_t min_marks_to_read, const std::size_t block_size,
|
||||
MergeTreeData & storage, const bool use_uncompressed_cache, const ExpressionActionsPtr & prewhere_actions,
|
||||
const String & prewhere_column, const Settings & settings, const Names & virt_column_names)
|
||||
: thread{thread}, pool{pool}, block_size_marks{block_size / storage.index_granularity},
|
||||
/// round min_marks_to_read up to nearest multiple of block_size expressed in marks
|
||||
min_marks_to_read{block_size
|
||||
? (min_marks_to_read * storage.index_granularity + block_size - 1)
|
||||
/ block_size * block_size / storage.index_granularity
|
||||
: min_marks_to_read
|
||||
},
|
||||
storage{storage}, use_uncompressed_cache{use_uncompressed_cache}, prewhere_actions{prewhere_actions},
|
||||
prewhere_column{prewhere_column}, min_bytes_to_use_direct_io{settings.min_bytes_to_use_direct_io},
|
||||
max_read_buffer_size{settings.max_read_buffer_size}, virt_column_names{virt_column_names},
|
||||
log{&Logger::get("MergeTreeThreadBlockInputStream")}
|
||||
{}
|
||||
|
||||
String getName() const override { return "MergeTreeThread"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
std::stringstream res;
|
||||
/// @todo print some meaningful information
|
||||
// res << "MergeTreeThread(columns";
|
||||
//
|
||||
// for (const auto & column : columns)
|
||||
// res << ", " << column.name;
|
||||
//
|
||||
// if (prewhere_actions)
|
||||
// res << ", prewhere, " << prewhere_actions->getID();
|
||||
//
|
||||
// res << ", marks";
|
||||
//
|
||||
// for (size_t i = 0; i < all_mark_ranges.size(); ++i)
|
||||
// res << ", " << all_mark_ranges[i].begin << ", " << all_mark_ranges[i].end;
|
||||
//
|
||||
// res << ")";
|
||||
return res.str();
|
||||
}
|
||||
|
||||
protected:
|
||||
/// Будем вызывать progressImpl самостоятельно.
|
||||
void progress(const Progress & value) override {}
|
||||
|
||||
Block readImpl() override
|
||||
{
|
||||
Block res;
|
||||
|
||||
while (!res)
|
||||
{
|
||||
if (!task && !getNewTask())
|
||||
break;
|
||||
|
||||
res = readFromPart();
|
||||
|
||||
if (res)
|
||||
injectVirtualColumns(res);
|
||||
|
||||
if (task->mark_ranges.empty())
|
||||
task = {};
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
private:
|
||||
/// Requests read task from MergeTreeReadPool and signals whether it got one
|
||||
bool getNewTask()
|
||||
{
|
||||
task = pool->getTask(min_marks_to_read, thread);
|
||||
|
||||
if (!task)
|
||||
{
|
||||
/** Закрываем файлы (ещё до уничтожения объекта).
|
||||
* Чтобы при создании многих источников, но одновременном чтении только из нескольких,
|
||||
* буферы не висели в памяти. */
|
||||
reader = {};
|
||||
pre_reader = {};
|
||||
return false;
|
||||
}
|
||||
|
||||
const auto path = storage.getFullPath() + task->data_part->name + '/';
|
||||
|
||||
if (!reader)
|
||||
{
|
||||
if (use_uncompressed_cache)
|
||||
owned_uncompressed_cache = storage.context.getUncompressedCache();
|
||||
|
||||
owned_mark_cache = storage.context.getMarkCache();
|
||||
|
||||
reader = std::make_unique<MergeTreeReader>(
|
||||
path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size);
|
||||
|
||||
if (prewhere_actions)
|
||||
pre_reader = std::make_unique<MergeTreeReader>(
|
||||
path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(),
|
||||
owned_mark_cache.get(), storage, task->mark_ranges, min_bytes_to_use_direct_io,
|
||||
max_read_buffer_size);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// retain avg_value_size_hints
|
||||
reader = std::make_unique<MergeTreeReader>(
|
||||
path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size,
|
||||
reader->getAvgValueSizeHints());
|
||||
|
||||
if (prewhere_actions)
|
||||
pre_reader = std::make_unique<MergeTreeReader>(
|
||||
path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(),
|
||||
owned_mark_cache.get(), storage, task->mark_ranges, min_bytes_to_use_direct_io,
|
||||
max_read_buffer_size, pre_reader->getAvgValueSizeHints());
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
Block readFromPart()
|
||||
{
|
||||
Block res;
|
||||
|
||||
if (prewhere_actions)
|
||||
{
|
||||
do
|
||||
{
|
||||
/// Прочитаем полный блок столбцов, нужных для вычисления выражения в PREWHERE.
|
||||
size_t space_left = std::max(1LU, block_size_marks);
|
||||
MarkRanges ranges_to_read;
|
||||
while (!task->mark_ranges.empty() && space_left)
|
||||
{
|
||||
auto & range = task->mark_ranges.back();
|
||||
|
||||
size_t marks_to_read = std::min(range.end - range.begin, space_left);
|
||||
pre_reader->readRange(range.begin, range.begin + marks_to_read, res);
|
||||
|
||||
ranges_to_read.emplace_back(range.begin, range.begin + marks_to_read);
|
||||
space_left -= marks_to_read;
|
||||
range.begin += marks_to_read;
|
||||
if (range.begin == range.end)
|
||||
task->mark_ranges.pop_back();
|
||||
}
|
||||
progressImpl({ res.rowsInFirstColumn(), res.bytes() });
|
||||
pre_reader->fillMissingColumns(res, task->ordered_names, task->should_reorder);
|
||||
|
||||
/// Вычислим выражение в PREWHERE.
|
||||
prewhere_actions->execute(res);
|
||||
|
||||
ColumnPtr column = res.getByName(prewhere_column).column;
|
||||
if (task->remove_prewhere_column)
|
||||
res.erase(prewhere_column);
|
||||
|
||||
const auto pre_bytes = res.bytes();
|
||||
|
||||
/** Если фильтр - константа (например, написано PREWHERE 1),
|
||||
* то либо вернём пустой блок, либо вернём блок без изменений.
|
||||
*/
|
||||
if (const auto column_const = typeid_cast<const ColumnConstUInt8 *>(column.get()))
|
||||
{
|
||||
if (!column_const->getData())
|
||||
{
|
||||
res.clear();
|
||||
return res;
|
||||
}
|
||||
|
||||
for (const auto & range : ranges_to_read)
|
||||
reader->readRange(range.begin, range.end, res);
|
||||
|
||||
progressImpl({ 0, res.bytes() - pre_bytes });
|
||||
}
|
||||
else if (const auto column_vec = typeid_cast<const ColumnUInt8 *>(column.get()))
|
||||
{
|
||||
size_t index_granularity = storage.index_granularity;
|
||||
|
||||
const auto & pre_filter = column_vec->getData();
|
||||
IColumn::Filter post_filter(pre_filter.size());
|
||||
|
||||
/// Прочитаем в нужных отрезках остальные столбцы и составим для них свой фильтр.
|
||||
size_t pre_filter_pos = 0;
|
||||
size_t post_filter_pos = 0;
|
||||
|
||||
for (const auto & range : ranges_to_read)
|
||||
{
|
||||
auto begin = range.begin;
|
||||
auto pre_filter_begin_pos = pre_filter_pos;
|
||||
|
||||
for (auto mark = range.begin; mark <= range.end; ++mark)
|
||||
{
|
||||
UInt8 nonzero = 0;
|
||||
|
||||
if (mark != range.end)
|
||||
{
|
||||
const size_t limit = std::min(pre_filter.size(), pre_filter_pos + index_granularity);
|
||||
for (size_t row = pre_filter_pos; row < limit; ++row)
|
||||
nonzero |= pre_filter[row];
|
||||
}
|
||||
|
||||
if (!nonzero)
|
||||
{
|
||||
if (mark > begin)
|
||||
{
|
||||
memcpy(
|
||||
&post_filter[post_filter_pos],
|
||||
&pre_filter[pre_filter_begin_pos],
|
||||
pre_filter_pos - pre_filter_begin_pos);
|
||||
post_filter_pos += pre_filter_pos - pre_filter_begin_pos;
|
||||
reader->readRange(begin, mark, res);
|
||||
}
|
||||
begin = mark + 1;
|
||||
pre_filter_begin_pos = std::min(pre_filter_pos + index_granularity, pre_filter.size());
|
||||
}
|
||||
|
||||
if (mark < range.end)
|
||||
pre_filter_pos = std::min(pre_filter_pos + index_granularity, pre_filter.size());
|
||||
}
|
||||
}
|
||||
|
||||
if (!post_filter_pos)
|
||||
{
|
||||
res.clear();
|
||||
continue;
|
||||
}
|
||||
|
||||
progressImpl({ 0, res.bytes() - pre_bytes });
|
||||
|
||||
post_filter.resize(post_filter_pos);
|
||||
|
||||
/// Отфильтруем столбцы, относящиеся к PREWHERE, используя pre_filter,
|
||||
/// остальные столбцы - используя post_filter.
|
||||
size_t rows = 0;
|
||||
for (const auto i : ext::range(0, res.columns()))
|
||||
{
|
||||
auto & col = res.getByPosition(i);
|
||||
if (col.name == prewhere_column && res.columns() > 1)
|
||||
continue;
|
||||
col.column =
|
||||
col.column->filter(task->column_name_set.count(col.name) ? post_filter : pre_filter);
|
||||
rows = col.column->size();
|
||||
}
|
||||
|
||||
/// Заменим столбец со значением условия из PREWHERE на константу.
|
||||
if (!task->remove_prewhere_column)
|
||||
res.getByName(prewhere_column).column = new ColumnConstUInt8{rows, 1};
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
"Illegal type " + column->getName() + " of column for filter. Must be ColumnUInt8 or ColumnConstUInt8.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER
|
||||
};
|
||||
|
||||
reader->fillMissingColumnsAndReorder(res, task->ordered_names);
|
||||
}
|
||||
while (!task->mark_ranges.empty() && !res && !isCancelled());
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t space_left = std::max(1LU, block_size_marks);
|
||||
|
||||
while (!task->mark_ranges.empty() && space_left)
|
||||
{
|
||||
auto & range = task->mark_ranges.back();
|
||||
|
||||
const size_t marks_to_read = std::min(range.end - range.begin, space_left);
|
||||
reader->readRange(range.begin, range.begin + marks_to_read, res);
|
||||
|
||||
space_left -= marks_to_read;
|
||||
range.begin += marks_to_read;
|
||||
if (range.begin == range.end)
|
||||
task->mark_ranges.pop_back();
|
||||
}
|
||||
|
||||
progressImpl({ res.rowsInFirstColumn(), res.bytes() });
|
||||
|
||||
reader->fillMissingColumns(res, task->ordered_names, task->should_reorder);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void injectVirtualColumns(Block & block)
|
||||
{
|
||||
const auto rows = block.rowsInFirstColumn();
|
||||
|
||||
/// add virtual columns
|
||||
if (!virt_column_names.empty())
|
||||
{
|
||||
for (const auto & virt_column_name : virt_column_names)
|
||||
{
|
||||
if (virt_column_name == "_part")
|
||||
{
|
||||
block.insert(ColumnWithTypeAndName{
|
||||
ColumnConst<String>{rows, task->data_part->name}.convertToFullColumn(),
|
||||
new DataTypeString,
|
||||
virt_column_name
|
||||
});
|
||||
}
|
||||
else if (virt_column_name == "_part_index")
|
||||
{
|
||||
block.insert(ColumnWithTypeAndName{
|
||||
ColumnConst<UInt64>{rows, task->part_index_in_query}.convertToFullColumn(),
|
||||
new DataTypeUInt64,
|
||||
virt_column_name
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeReadPoolPtr pool;
|
||||
const std::size_t block_size_marks;
|
||||
const std::size_t min_marks_to_read;
|
||||
MergeTreeData & storage;
|
||||
const bool use_uncompressed_cache;
|
||||
ExpressionActionsPtr prewhere_actions;
|
||||
const String prewhere_column;
|
||||
const std::size_t min_bytes_to_use_direct_io;
|
||||
const std::size_t max_read_buffer_size;
|
||||
const Names virt_column_names;
|
||||
|
||||
Logger * log;
|
||||
|
||||
using MergeTreeReaderPtr = std::unique_ptr<MergeTreeReader>;
|
||||
|
||||
UncompressedCachePtr owned_uncompressed_cache;
|
||||
MarkCachePtr owned_mark_cache;
|
||||
|
||||
MergeTreeReadTaskPtr task;
|
||||
MergeTreeReaderPtr reader;
|
||||
MergeTreeReaderPtr pre_reader;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -33,7 +33,8 @@ namespace DB
|
||||
class MergeTreeWhereOptimizer
|
||||
{
|
||||
static constexpr auto threshold = 10;
|
||||
static constexpr auto max_columns_relative_size = 0.25f;
|
||||
/// Решили убрать ограничение в виду отсутствия штрафа по скорости на перенос в PREWHERE
|
||||
static constexpr auto max_columns_relative_size = 1.0f;
|
||||
static constexpr auto and_function_name = "and";
|
||||
static constexpr auto equals_function_name = "equals";
|
||||
static constexpr auto array_join_function_name = "arrayJoin";
|
||||
|
29
dbms/include/DB/Storages/MergeTree/RangesInDataPart.h
Normal file
29
dbms/include/DB/Storages/MergeTree/RangesInDataPart.h
Normal file
@ -0,0 +1,29 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DB/Storages/MergeTree/MarkRange.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
struct RangesInDataPart
|
||||
{
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
std::size_t part_index_in_query;
|
||||
MarkRanges ranges;
|
||||
|
||||
RangesInDataPart() = default;
|
||||
|
||||
RangesInDataPart(const MergeTreeData::DataPartPtr & data_part, const std::size_t part_index_in_query,
|
||||
const MarkRanges & ranges = MarkRanges{})
|
||||
: data_part{data_part}, part_index_in_query{part_index_in_query}, ranges{ranges}
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
using RangesInDataParts = std::vector<RangesInDataPart>;
|
||||
|
||||
|
||||
}
|
@ -2,7 +2,9 @@
|
||||
|
||||
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
||||
#include <DB/Storages/MergeTree/AbandonableLockInZooKeeper.h>
|
||||
#include <DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -11,9 +13,14 @@ namespace DB
|
||||
class ReplicatedMergeTreeBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
ReplicatedMergeTreeBlockOutputStream(StorageReplicatedMergeTree & storage_, const String & insert_id_)
|
||||
: storage(storage_), insert_id(insert_id_),
|
||||
log(&Logger::get(storage.data.getLogName() + " (Replicated OutputStream)")) {}
|
||||
ReplicatedMergeTreeBlockOutputStream(StorageReplicatedMergeTree & storage_, const String & insert_id_, size_t quorum_)
|
||||
: storage(storage_), insert_id(insert_id_), quorum(quorum_),
|
||||
log(&Logger::get(storage.data.getLogName() + " (Replicated OutputStream)"))
|
||||
{
|
||||
/// Значение кворума 1 имеет такой же смысл, как если он отключён.
|
||||
if (quorum == 1)
|
||||
quorum = 0;
|
||||
}
|
||||
|
||||
void writePrefix() override
|
||||
{
|
||||
@ -26,6 +33,38 @@ public:
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
assertSessionIsNotExpired(zookeeper);
|
||||
|
||||
/** Если запись с кворумом, то проверим, что требуемое количество реплик сейчас живо,
|
||||
* а также что для всех предыдущих кусков, для которых требуется кворум, этот кворум достигнут.
|
||||
*/
|
||||
String quorum_status_path = storage.zookeeper_path + "/quorum/status";
|
||||
if (quorum)
|
||||
{
|
||||
/// Список живых реплик. Все они регистрируют эфемерную ноду для leader_election.
|
||||
auto live_replicas = zookeeper->getChildren(storage.zookeeper_path + "/leader_election");
|
||||
|
||||
if (live_replicas.size() < quorum)
|
||||
throw Exception("Number of alive replicas ("
|
||||
+ toString(live_replicas.size()) + ") is less than requested quorum (" + toString(quorum) + ").",
|
||||
ErrorCodes::TOO_LESS_LIVE_REPLICAS);
|
||||
|
||||
/** Достигнут ли кворум для последнего куска, для которого нужен кворум?
|
||||
* Запись всех кусков с включенным кворумом линейно упорядочена.
|
||||
* Это значит, что в любой момент времени может быть только один кусок,
|
||||
* для которого нужен, но ещё не достигнут кворум.
|
||||
* Информация о таком куске будет расположена в ноде /quorum/status.
|
||||
* Если кворум достигнут, то нода удаляется.
|
||||
*/
|
||||
|
||||
String quorum_status;
|
||||
bool quorum_unsatisfied = zookeeper->tryGet(quorum_status_path, quorum_status);
|
||||
|
||||
if (quorum_unsatisfied)
|
||||
throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status, ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE);
|
||||
|
||||
/// Обе проверки неявно делаются и позже (иначе был бы race condition).
|
||||
}
|
||||
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block);
|
||||
|
||||
for (auto & current_block : part_blocks)
|
||||
@ -47,48 +86,96 @@ public:
|
||||
/// NOTE: Если такая дедупликация не нужна, можно вместо этого оставлять block_id пустым.
|
||||
/// Можно для этого сделать настройку или синтаксис в запросе (например, ID=null).
|
||||
if (block_id.empty())
|
||||
{
|
||||
block_id = part->checksums.summaryDataChecksum();
|
||||
|
||||
if (block_id.empty())
|
||||
throw Exception("Logical error: block_id is empty.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Wrote block " << part_number << " with ID " << block_id << ", " << current_block.block.rows() << " rows");
|
||||
|
||||
StorageReplicatedMergeTree::LogEntry log_entry;
|
||||
log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART;
|
||||
log_entry.source_replica = storage.replica_name;
|
||||
log_entry.new_part_name = part_name;
|
||||
log_entry.quorum = quorum;
|
||||
|
||||
/// Одновременно добавим информацию о куске во все нужные места в ZooKeeper и снимем block_number_lock.
|
||||
|
||||
/// Информация о блоке.
|
||||
zkutil::Ops ops;
|
||||
if (!block_id.empty())
|
||||
{
|
||||
ops.push_back(new zkutil::Op::Create(
|
||||
|
||||
ops.push_back(
|
||||
new zkutil::Op::Create(
|
||||
storage.zookeeper_path + "/blocks/" + block_id,
|
||||
"",
|
||||
zookeeper->getDefaultACL(),
|
||||
zkutil::CreateMode::Persistent));
|
||||
ops.push_back(new zkutil::Op::Create(
|
||||
ops.push_back(
|
||||
new zkutil::Op::Create(
|
||||
storage.zookeeper_path + "/blocks/" + block_id + "/columns",
|
||||
part->columns.toString(),
|
||||
zookeeper->getDefaultACL(),
|
||||
zkutil::CreateMode::Persistent));
|
||||
ops.push_back(new zkutil::Op::Create(
|
||||
ops.push_back(
|
||||
new zkutil::Op::Create(
|
||||
storage.zookeeper_path + "/blocks/" + block_id + "/checksums",
|
||||
part->checksums.toString(),
|
||||
zookeeper->getDefaultACL(),
|
||||
zkutil::CreateMode::Persistent));
|
||||
ops.push_back(new zkutil::Op::Create(
|
||||
ops.push_back(
|
||||
new zkutil::Op::Create(
|
||||
storage.zookeeper_path + "/blocks/" + block_id + "/number",
|
||||
toString(part_number),
|
||||
zookeeper->getDefaultACL(),
|
||||
zkutil::CreateMode::Persistent));
|
||||
}
|
||||
|
||||
/// Информация о куске, в данных реплики.
|
||||
storage.checkPartAndAddToZooKeeper(part, ops, part_name);
|
||||
|
||||
/// Лог репликации.
|
||||
ops.push_back(new zkutil::Op::Create(
|
||||
storage.zookeeper_path + "/log/log-",
|
||||
log_entry.toString(),
|
||||
zookeeper->getDefaultACL(),
|
||||
zkutil::CreateMode::PersistentSequential));
|
||||
|
||||
/// Удаление информации о том, что номер блока используется для записи.
|
||||
block_number_lock.getUnlockOps(ops);
|
||||
|
||||
/** Если нужен кворум - создание узла, в котором отслеживается кворум.
|
||||
* (Если такой узел уже существует - значит кто-то успел одновременно сделать другую кворумную запись, но для неё кворум ещё не достигнут.
|
||||
* Делать в это время следующую кворумную запись нельзя.)
|
||||
*/
|
||||
if (quorum)
|
||||
{
|
||||
static std::once_flag once_flag;
|
||||
std::call_once(once_flag, [&]
|
||||
{
|
||||
zookeeper->createIfNotExists(storage.zookeeper_path + "/quorum", "");
|
||||
});
|
||||
|
||||
ReplicatedMergeTreeQuorumEntry quorum_entry;
|
||||
quorum_entry.part_name = part_name;
|
||||
quorum_entry.required_number_of_replicas = quorum;
|
||||
quorum_entry.replicas.insert(storage.replica_name);
|
||||
|
||||
/** В данный момент, этот узел будет содержать информацию о том, что текущая реплика получила кусок.
|
||||
* Когда другие реплики будут получать этот кусок (обычным способом, обрабатывая лог репликации),
|
||||
* они будут добавлять себя в содержимое этого узла.
|
||||
* Когда в нём будет информация о quorum количестве реплик, этот узел удаляется,
|
||||
* что говорит о том, что кворум достигнут.
|
||||
*/
|
||||
|
||||
ops.push_back(
|
||||
new zkutil::Op::Create(
|
||||
quorum_status_path,
|
||||
quorum_entry.toString(),
|
||||
zookeeper->getDefaultACL(),
|
||||
zkutil::CreateMode::Persistent));
|
||||
}
|
||||
|
||||
MergeTreeData::Transaction transaction; /// Если не получится добавить кусок в ZK, снова уберем его из рабочего набора.
|
||||
storage.data.renameTempPartAndAdd(part, nullptr, &transaction);
|
||||
|
||||
@ -113,9 +200,19 @@ public:
|
||||
|
||||
/// Если данные отличались от тех, что были вставлены ранее с тем же ID, бросим исключение.
|
||||
expected_checksums.checkEqual(part->checksums, true);
|
||||
|
||||
transaction.rollback();
|
||||
}
|
||||
else if (zookeeper->exists(quorum_status_path))
|
||||
{
|
||||
transaction.rollback();
|
||||
|
||||
throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Сюда можем попасть также, если узел с кворумом существовал, но потом быстро был удалён.
|
||||
|
||||
throw Exception("Unexpected ZNODEEXISTS while adding block " + toString(part_number) + " with ID " + block_id + ": "
|
||||
+ zkutil::ZooKeeper::error2string(code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR);
|
||||
}
|
||||
@ -140,12 +237,21 @@ public:
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
if (quorum)
|
||||
{
|
||||
/// Дожидаемся достижения кворума. TODO Настраиваемый таймаут.
|
||||
LOG_TRACE(log, "Waiting for quorum");
|
||||
zookeeper->waitForDisappear(quorum_status_path);
|
||||
LOG_TRACE(log, "Quorum satisfied");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
StorageReplicatedMergeTree & storage;
|
||||
String insert_id;
|
||||
size_t quorum;
|
||||
size_t block_index = 0;
|
||||
|
||||
Logger * log;
|
||||
|
@ -3,10 +3,14 @@
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include <DB/Core/Types.h>
|
||||
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
|
||||
|
||||
struct Stat;
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -67,6 +71,8 @@ struct ReplicatedMergeTreeLogEntry
|
||||
/// Время создания или время копирования из общего лога в очередь конкретной реплики.
|
||||
time_t create_time = 0;
|
||||
|
||||
/// Величина кворума (для GET_PART) - ненулевое значение при включенной кворумной записи.
|
||||
size_t quorum = 0;
|
||||
|
||||
void addResultToVirtualParts(StorageReplicatedMergeTree & storage);
|
||||
void tagPartAsFuture(StorageReplicatedMergeTree & storage);
|
||||
@ -75,7 +81,7 @@ struct ReplicatedMergeTreeLogEntry
|
||||
void readText(ReadBuffer & in);
|
||||
|
||||
String toString() const;
|
||||
static Ptr parse(const String & s);
|
||||
static Ptr parse(const String & s, const Stat & stat);
|
||||
};
|
||||
|
||||
|
||||
|
@ -0,0 +1,72 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Types.h>
|
||||
#include <DB/IO/ReadBuffer.h>
|
||||
#include <DB/IO/ReadBufferFromString.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
#include <DB/IO/Operators.h>
|
||||
#include <zkutil/ZooKeeper.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Для реализации функциональности "кворумная запись".
|
||||
* Информация о том, на каких репликах появился вставленный кусок данных,
|
||||
* и на скольких репликах он должен быть.
|
||||
*/
|
||||
struct ReplicatedMergeTreeQuorumEntry
|
||||
{
|
||||
String part_name;
|
||||
size_t required_number_of_replicas;
|
||||
std::set<String> replicas;
|
||||
|
||||
void writeText(WriteBuffer & out) const
|
||||
{
|
||||
out << "version: 1\n"
|
||||
<< "part_name: " << part_name << "\n"
|
||||
<< "required_number_of_replicas: " << required_number_of_replicas << "\n"
|
||||
<< "actual_number_of_replicas: " << replicas.size() << "\n"
|
||||
<< "replicas:\n";
|
||||
|
||||
for (const auto & replica : replicas)
|
||||
out << escape << replica << "\n";
|
||||
}
|
||||
|
||||
void readText(ReadBuffer & in)
|
||||
{
|
||||
size_t actual_number_of_replicas = 0;
|
||||
|
||||
in >> "version: 1\n"
|
||||
>> "part_name: " >> part_name >> "\n"
|
||||
>> "required_number_of_replicas: " >> required_number_of_replicas >> "\n"
|
||||
>> "actual_number_of_replicas: " >> actual_number_of_replicas >> "\n"
|
||||
>> "replicas:\n";
|
||||
|
||||
for (size_t i = 0; i < actual_number_of_replicas; ++i)
|
||||
{
|
||||
String replica;
|
||||
in >> escape >> replica >> "\n";
|
||||
replicas.insert(replica);
|
||||
}
|
||||
}
|
||||
|
||||
String toString() const
|
||||
{
|
||||
String res;
|
||||
{
|
||||
WriteBufferFromString out(res);
|
||||
writeText(out);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
void fromString(const String & str)
|
||||
{
|
||||
ReadBufferFromString in(str);
|
||||
readText(in);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -70,7 +70,7 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
/// Сбрасывает все буферы в подчинённую таблицу.
|
||||
void shutdown() override;
|
||||
|
@ -59,7 +59,7 @@ public:
|
||||
const std::string & chunk_name);
|
||||
|
||||
/// Если бы запись была разрешена, непонятно, как назвать новый чанк.
|
||||
BlockOutputStreamPtr write(ASTPtr query) override
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override
|
||||
{
|
||||
throw Exception("Table doesn't support writing", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
void drop() override {}
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
|
||||
@ -78,7 +78,10 @@ public:
|
||||
|
||||
const ExpressionActionsPtr & getShardingKeyExpr() const { return sharding_key_expr; }
|
||||
const String & getShardingKeyColumnName() const { return sharding_key_column_name; }
|
||||
size_t getShardCount() const;
|
||||
const String & getPath() const { return path; }
|
||||
std::string getRemoteDatabaseName() const { return remote_database; }
|
||||
std::string getRemoteTableName() const { return remote_table; }
|
||||
|
||||
|
||||
private:
|
||||
|
@ -68,7 +68,7 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
|
||||
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
bool supportsIndexForIn() const override { return data->supportsIndexForIn(); }
|
||||
bool supportsParallelReplicas() const override { return data->supportsParallelReplicas(); }
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
void drop() override;
|
||||
bool optimize(const Settings & settings) override;
|
||||
|
||||
|
@ -51,7 +51,7 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
void drop() override;
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
|
||||
|
@ -80,7 +80,7 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
/** Выполнить очередной шаг объединения кусков.
|
||||
*/
|
||||
|
@ -42,7 +42,7 @@ public:
|
||||
return { new NullBlockInputStream };
|
||||
}
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override
|
||||
{
|
||||
return new NullBlockOutputStream;
|
||||
}
|
||||
|
@ -82,7 +82,7 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
bool optimize(const Settings & settings) override;
|
||||
|
||||
@ -368,8 +368,11 @@ private:
|
||||
void alterThread();
|
||||
|
||||
/** Проверяет целостность кусков.
|
||||
* Находит отсутствующие куски.
|
||||
*/
|
||||
void partCheckThread();
|
||||
void checkPart(const String & part_name);
|
||||
void searchForMissingPart(const String & part_name);
|
||||
|
||||
/// Обмен кусками.
|
||||
|
||||
@ -379,8 +382,9 @@ private:
|
||||
|
||||
/** Скачать указанный кусок с указанной реплики.
|
||||
* Если to_detached, то кусок помещается в директорию detached.
|
||||
* Если quorum != 0, то обновляется узел для отслеживания кворума.
|
||||
*/
|
||||
void fetchPart(const String & part_name, const String & replica_path, bool to_detached = false);
|
||||
void fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum);
|
||||
|
||||
AbandonableLockInZooKeeper allocateBlockNumber(const String & month_name);
|
||||
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
protected:
|
||||
StorageSetOrJoinBase(
|
||||
|
@ -48,7 +48,7 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
|
||||
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1) override;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
|
||||
|
||||
void drop() override;
|
||||
|
||||
|
@ -129,7 +129,7 @@ private:
|
||||
NamesAndTypesListPtr chooseColumns(Cluster & cluster, const String & database, const String & table, const Context & context) const
|
||||
{
|
||||
/// Запрос на описание таблицы
|
||||
String query = "DESC TABLE " + database + "." + table;
|
||||
String query = "DESC TABLE " + backQuoteIfNeed(database) + "." + backQuoteIfNeed(table);
|
||||
Settings settings = context.getSettings();
|
||||
NamesAndTypesList res;
|
||||
|
||||
|
262
dbms/scripts/gen-bias-data.py
Executable file
262
dbms/scripts/gen-bias-data.py
Executable file
@ -0,0 +1,262 @@
|
||||
#!/usr/bin/python3.4
|
||||
# -*- coding: utf-8 -*-
|
||||
|
||||
import sys
|
||||
import argparse
|
||||
import tempfile
|
||||
import random
|
||||
import subprocess
|
||||
import bisect
|
||||
from copy import deepcopy
|
||||
|
||||
# Псевдослучайный генератор уникальных чисел.
|
||||
# http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/
|
||||
class UniqueRandomGenerator:
|
||||
prime = 4294967291
|
||||
|
||||
def __init__(self, seed_base, seed_offset):
|
||||
self.index = self.permutePQR(self.permutePQR(seed_base) + 0x682f0161)
|
||||
self.intermediate_offset = self.permutePQR(self.permutePQR(seed_offset) + 0x46790905)
|
||||
|
||||
def next(self):
|
||||
val = self.permutePQR((self.permutePQR(self.index) + self.intermediate_offset) ^ 0x5bf03635)
|
||||
self.index = self.index + 1
|
||||
return val
|
||||
|
||||
def permutePQR(self, x):
|
||||
if x >=self.prime:
|
||||
return x
|
||||
else:
|
||||
residue = (x * x) % self.prime
|
||||
if x <= self.prime/2:
|
||||
return residue
|
||||
else:
|
||||
return self.prime - residue
|
||||
|
||||
# Создать таблицу содержащую уникальные значения.
|
||||
def generate_data_source(host, port, http_port, min_cardinality, max_cardinality, count):
|
||||
chunk_size = round((max_cardinality - min_cardinality) / float(count))
|
||||
used_values = 0
|
||||
|
||||
cur_count = 0
|
||||
next_size = 0
|
||||
|
||||
sup = 32768
|
||||
n1 = random.randrange(0, sup)
|
||||
n2 = random.randrange(0, sup)
|
||||
urng = UniqueRandomGenerator(n1, n2)
|
||||
|
||||
is_first = True
|
||||
|
||||
with tempfile.TemporaryDirectory() as tmp_dir:
|
||||
filename = tmp_dir + '/table.txt'
|
||||
with open(filename, 'w+b') as file_handle:
|
||||
while cur_count < count:
|
||||
|
||||
if is_first == True:
|
||||
is_first = False
|
||||
if min_cardinality != 0:
|
||||
next_size = min_cardinality + 1
|
||||
else:
|
||||
next_size = chunk_size
|
||||
else:
|
||||
next_size += chunk_size
|
||||
|
||||
while used_values < next_size:
|
||||
h = urng.next()
|
||||
used_values = used_values + 1
|
||||
out = str(h) + "\t" + str(cur_count) + "\n";
|
||||
file_handle.write(bytes(out, 'UTF-8'));
|
||||
cur_count = cur_count + 1
|
||||
|
||||
query = "DROP TABLE IF EXISTS data_source"
|
||||
subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query])
|
||||
query = "CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog"
|
||||
subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query])
|
||||
|
||||
cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE)
|
||||
subprocess.check_output(("POST", "http://{0}:{1}/?query=INSERT INTO data_source FORMAT TabSeparated".format(host, http_port)), stdin=cat.stdout)
|
||||
cat.wait()
|
||||
|
||||
def perform_query(host, port):
|
||||
query = "SELECT runningAccumulate(uniqExactState(UserID)) AS exact, "
|
||||
query += "runningAccumulate(uniqCombinedRawState(UserID)) AS approx "
|
||||
query += "FROM data_source GROUP BY KeyID"
|
||||
return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query])
|
||||
|
||||
def parse_clickhouse_response(response):
|
||||
parsed = []
|
||||
lines = response.decode().split("\n")
|
||||
for cur_line in lines:
|
||||
rows = cur_line.split("\t")
|
||||
if len(rows) == 2:
|
||||
parsed.append([float(rows[0]), float(rows[1])])
|
||||
return parsed
|
||||
|
||||
def accumulate_data(accumulated_data, data):
|
||||
if not accumulated_data:
|
||||
accumulated_data = deepcopy(data)
|
||||
else:
|
||||
for row1, row2 in zip(accumulated_data, data):
|
||||
row1[1] += row2[1];
|
||||
return accumulated_data
|
||||
|
||||
def generate_raw_result(accumulated_data, count):
|
||||
expected_tab = []
|
||||
bias_tab = []
|
||||
for row in accumulated_data:
|
||||
exact = row[0]
|
||||
expected = row[1] / count
|
||||
bias = expected - exact
|
||||
|
||||
expected_tab.append(expected)
|
||||
bias_tab.append(bias)
|
||||
return [ expected_tab, bias_tab ]
|
||||
|
||||
def generate_sample(raw_estimates, biases, n_samples):
|
||||
result = []
|
||||
|
||||
min_card = raw_estimates[0]
|
||||
max_card = raw_estimates[len(raw_estimates) - 1]
|
||||
step = (max_card - min_card) / (n_samples - 1)
|
||||
|
||||
for i in range(0, n_samples + 1):
|
||||
x = min_card + i * step
|
||||
j = bisect.bisect_left(raw_estimates, x)
|
||||
|
||||
if j == len(raw_estimates):
|
||||
result.append((raw_estimates[j - 1], biases[j - 1]))
|
||||
elif raw_estimates[j] == x:
|
||||
result.append((raw_estimates[j], biases[j]))
|
||||
else:
|
||||
# Найти 6 ближайших соседей. Вычислить среднее арифметическое.
|
||||
|
||||
# 6 точек слева x [j-6 j-5 j-4 j-3 j-2 j-1]
|
||||
|
||||
begin = max(j - 6, 0) - 1
|
||||
end = j - 1
|
||||
|
||||
T = []
|
||||
for k in range(end, begin, -1):
|
||||
T.append(x - raw_estimates[k])
|
||||
|
||||
# 6 точек справа x [j j+1 j+2 j+3 j+4 j+5]
|
||||
|
||||
begin = j
|
||||
end = min(j + 5, len(raw_estimates) - 1) + 1
|
||||
|
||||
U = []
|
||||
for k in range(begin, end):
|
||||
U.append(raw_estimates[k] - x)
|
||||
|
||||
# Сливаем расстояния.
|
||||
|
||||
V = []
|
||||
|
||||
lim = min(len(T), len(U))
|
||||
k1 = 0
|
||||
k2 = 0
|
||||
|
||||
while k1 < lim and k2 < lim:
|
||||
if T[k1] == U[k2]:
|
||||
V.append(j - k1 - 1)
|
||||
V.append(j + k2)
|
||||
k1 = k1 + 1
|
||||
k2 = k2 + 1
|
||||
elif T[k1] < U[k2]:
|
||||
V.append(j - k1 - 1)
|
||||
k1 = k1 + 1
|
||||
else:
|
||||
V.append(j + k2)
|
||||
k2 = k2 + 1
|
||||
|
||||
if k1 < len(T):
|
||||
while k1 < len(T):
|
||||
V.append(j - k1 - 1)
|
||||
k1 = k1 + 1
|
||||
elif k2 < len(U):
|
||||
while k2 < len(U):
|
||||
V.append(j + k2)
|
||||
k2 = k2 + 1
|
||||
|
||||
# Выбираем 6 ближайших точек.
|
||||
# Вычисляем средние.
|
||||
|
||||
begin = 0
|
||||
end = min(len(V), 6)
|
||||
|
||||
sum = 0
|
||||
bias = 0
|
||||
for k in range(begin, end):
|
||||
sum += raw_estimates[V[k]]
|
||||
bias += biases[V[k]]
|
||||
sum /= float(end)
|
||||
bias /= float(end)
|
||||
|
||||
result.append((sum, bias))
|
||||
|
||||
# Пропустить последовательные результаты, чьи оценки одинаковые.
|
||||
final_result = []
|
||||
last = -1
|
||||
for entry in result:
|
||||
if entry[0] != last:
|
||||
final_result.append((entry[0], entry[1]))
|
||||
last = entry[0]
|
||||
|
||||
return final_result
|
||||
|
||||
def dump_arrays(data):
|
||||
|
||||
print("Size of each array: {0}\n".format(len(data)))
|
||||
|
||||
is_first = True
|
||||
sep = ''
|
||||
|
||||
print("raw_estimates = ")
|
||||
print("{")
|
||||
for row in data:
|
||||
print("\t{0}{1}".format(sep, row[0]))
|
||||
if is_first == True:
|
||||
is_first = False
|
||||
sep = ","
|
||||
print("};")
|
||||
|
||||
is_first = True
|
||||
sep = ""
|
||||
|
||||
print("\nbiases = ")
|
||||
print("{")
|
||||
for row in data:
|
||||
print("\t{0}{1}".format(sep, row[1]))
|
||||
if is_first == True:
|
||||
is_first = False
|
||||
sep = ","
|
||||
print("};")
|
||||
|
||||
def start():
|
||||
parser = argparse.ArgumentParser(description = "Generate bias correction tables for HyperLogLog-based functions.")
|
||||
parser.add_argument("-x", "--host", default="127.0.0.1", help="ClickHouse server host name");
|
||||
parser.add_argument("-p", "--port", type=int, default=9000, help="ClickHouse server TCP port");
|
||||
parser.add_argument("-t", "--http_port", type=int, default=8123, help="ClickHouse server HTTP port");
|
||||
parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations");
|
||||
parser.add_argument("-m", "--min_cardinality", type=int, default=16384, help="minimal cardinality");
|
||||
parser.add_argument("-M", "--max_cardinality", type=int, default=655360, help="maximal cardinality");
|
||||
parser.add_argument("-s", "--samples", type=int, default=200, help="number of sampled values");
|
||||
args = parser.parse_args()
|
||||
|
||||
accumulated_data = []
|
||||
|
||||
for i in range(0, args.iterations):
|
||||
print(i + 1)
|
||||
sys.stdout.flush()
|
||||
|
||||
generate_data_source(args.host, str(args.port), str(args.http_port), args.min_cardinality, args.max_cardinality, 1000)
|
||||
response = perform_query(args.host, str(args.port))
|
||||
data = parse_clickhouse_response(response)
|
||||
accumulated_data = accumulate_data(accumulated_data, data)
|
||||
|
||||
result = generate_raw_result(accumulated_data, args.iterations)
|
||||
sampled_data = generate_sample(result[0], result[1], args.samples)
|
||||
dump_arrays(sampled_data)
|
||||
|
||||
if __name__ == "__main__": start()
|
150
dbms/scripts/linear-counting-threshold.py
Executable file
150
dbms/scripts/linear-counting-threshold.py
Executable file
@ -0,0 +1,150 @@
|
||||
#!/usr/bin/python3.4
|
||||
# -*- coding: utf-8 -*-
|
||||
|
||||
import sys
|
||||
import argparse
|
||||
import tempfile
|
||||
import random
|
||||
import subprocess
|
||||
import bisect
|
||||
from copy import deepcopy
|
||||
|
||||
# Псевдослучайный генератор уникальных чисел.
|
||||
# http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/
|
||||
class UniqueRandomGenerator:
|
||||
prime = 4294967291
|
||||
|
||||
def __init__(self, seed_base, seed_offset):
|
||||
self.index = self.permutePQR(self.permutePQR(seed_base) + 0x682f0161)
|
||||
self.intermediate_offset = self.permutePQR(self.permutePQR(seed_offset) + 0x46790905)
|
||||
|
||||
def next(self):
|
||||
val = self.permutePQR((self.permutePQR(self.index) + self.intermediate_offset) ^ 0x5bf03635)
|
||||
self.index = self.index + 1
|
||||
return val
|
||||
|
||||
def permutePQR(self, x):
|
||||
if x >=self.prime:
|
||||
return x
|
||||
else:
|
||||
residue = (x * x) % self.prime
|
||||
if x <= self.prime/2:
|
||||
return residue
|
||||
else:
|
||||
return self.prime - residue
|
||||
|
||||
# Создать таблицу содержащую уникальные значения.
|
||||
def generate_data_source(host, port, http_port, min_cardinality, max_cardinality, count):
|
||||
chunk_size = round((max_cardinality - (min_cardinality + 1)) / float(count))
|
||||
used_values = 0
|
||||
|
||||
cur_count = 0
|
||||
next_size = 0
|
||||
|
||||
sup = 32768
|
||||
n1 = random.randrange(0, sup)
|
||||
n2 = random.randrange(0, sup)
|
||||
urng = UniqueRandomGenerator(n1, n2)
|
||||
|
||||
is_first = True
|
||||
|
||||
with tempfile.TemporaryDirectory() as tmp_dir:
|
||||
filename = tmp_dir + '/table.txt'
|
||||
with open(filename, 'w+b') as file_handle:
|
||||
while cur_count < count:
|
||||
|
||||
if is_first == True:
|
||||
is_first = False
|
||||
if min_cardinality != 0:
|
||||
next_size = min_cardinality + 1
|
||||
else:
|
||||
next_size = chunk_size
|
||||
else:
|
||||
next_size += chunk_size
|
||||
|
||||
while used_values < next_size:
|
||||
h = urng.next()
|
||||
used_values = used_values + 1
|
||||
out = str(h) + "\t" + str(cur_count) + "\n";
|
||||
file_handle.write(bytes(out, 'UTF-8'));
|
||||
cur_count = cur_count + 1
|
||||
|
||||
query = "DROP TABLE IF EXISTS data_source"
|
||||
subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query])
|
||||
query = "CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog"
|
||||
subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query])
|
||||
|
||||
cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE)
|
||||
subprocess.check_output(("POST", "http://{0}:{1}/?query=INSERT INTO data_source FORMAT TabSeparated".format(host, http_port)), stdin=cat.stdout)
|
||||
cat.wait()
|
||||
|
||||
def perform_query(host, port):
|
||||
query = "SELECT runningAccumulate(uniqExactState(UserID)) AS exact, "
|
||||
query += "runningAccumulate(uniqCombinedRawState(UserID)) AS raw, "
|
||||
query += "runningAccumulate(uniqCombinedLinearCountingState(UserID)) AS linear_counting, "
|
||||
query += "runningAccumulate(uniqCombinedBiasCorrectedState(UserID)) AS bias_corrected "
|
||||
query += "FROM data_source GROUP BY KeyID"
|
||||
return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query])
|
||||
|
||||
def parse_clickhouse_response(response):
|
||||
parsed = []
|
||||
lines = response.decode().split("\n")
|
||||
for cur_line in lines:
|
||||
rows = cur_line.split("\t")
|
||||
if len(rows) == 4:
|
||||
parsed.append([float(rows[0]), float(rows[1]), float(rows[2]), float(rows[3])])
|
||||
return parsed
|
||||
|
||||
def accumulate_data(accumulated_data, data):
|
||||
if not accumulated_data:
|
||||
accumulated_data = deepcopy(data)
|
||||
else:
|
||||
for row1, row2 in zip(accumulated_data, data):
|
||||
row1[1] += row2[1];
|
||||
row1[2] += row2[2];
|
||||
row1[3] += row2[3];
|
||||
return accumulated_data
|
||||
|
||||
def dump_graphs(data, count):
|
||||
with open("raw_graph.txt", "w+b") as fh1, open("linear_counting_graph.txt", "w+b") as fh2, open("bias_corrected_graph.txt", "w+b") as fh3:
|
||||
expected_tab = []
|
||||
bias_tab = []
|
||||
for row in data:
|
||||
exact = row[0]
|
||||
raw = row[1] / count;
|
||||
linear_counting = row[2] / count;
|
||||
bias_corrected = row[3] / count;
|
||||
|
||||
outstr = "{0}\t{1}\n".format(exact, abs(raw - exact) / exact)
|
||||
fh1.write(bytes(outstr, 'UTF-8'))
|
||||
|
||||
outstr = "{0}\t{1}\n".format(exact, abs(linear_counting - exact) / exact)
|
||||
fh2.write(bytes(outstr, 'UTF-8'))
|
||||
|
||||
outstr = "{0}\t{1}\n".format(exact, abs(bias_corrected - exact) / exact)
|
||||
fh3.write(bytes(outstr, 'UTF-8'))
|
||||
|
||||
def start():
|
||||
parser = argparse.ArgumentParser(description = "Generate graphs that help to determine the linear counting threshold.")
|
||||
parser.add_argument("-x", "--host", default="127.0.0.1", help="clickhouse host name");
|
||||
parser.add_argument("-p", "--port", type=int, default=9000, help="clickhouse client TCP port");
|
||||
parser.add_argument("-t", "--http_port", type=int, default=8123, help="clickhouse HTTP port");
|
||||
parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations");
|
||||
parser.add_argument("-m", "--min_cardinality", type=int, default=16384, help="minimal cardinality");
|
||||
parser.add_argument("-M", "--max_cardinality", type=int, default=655360, help="maximal cardinality");
|
||||
args = parser.parse_args()
|
||||
|
||||
accumulated_data = []
|
||||
|
||||
for i in range(0, args.iterations):
|
||||
print(i + 1)
|
||||
sys.stdout.flush()
|
||||
|
||||
generate_data_source(args.host, str(args.port), str(args.http_port), args.min_cardinality, args.max_cardinality, 1000)
|
||||
response = perform_query(args.host, str(args.port))
|
||||
data = parse_clickhouse_response(response)
|
||||
accumulated_data = accumulate_data(accumulated_data, data)
|
||||
|
||||
dump_graphs(accumulated_data, args.iterations)
|
||||
|
||||
if __name__ == "__main__": start()
|
@ -351,6 +351,72 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "uniqCombinedRaw")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const IDataType & argument_type = *argument_types[0];
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqCombinedRawData>(*argument_types[0]);
|
||||
|
||||
if (res)
|
||||
return res;
|
||||
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqCombinedRawData<DataTypeDate::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqCombinedRawData<DataTypeDateTime::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedRawData<String>>;
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "uniqCombinedLinearCounting")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const IDataType & argument_type = *argument_types[0];
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq,
|
||||
AggregateFunctionUniqCombinedLinearCountingData>(*argument_types[0]);
|
||||
|
||||
if (res)
|
||||
return res;
|
||||
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDate::FieldType,
|
||||
AggregateFunctionUniqCombinedLinearCountingData<DataTypeDate::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDateTime::FieldType,
|
||||
AggregateFunctionUniqCombinedLinearCountingData<DataTypeDateTime::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedLinearCountingData<String>>;
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "uniqCombinedBiasCorrected")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const IDataType & argument_type = *argument_types[0];
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq,
|
||||
AggregateFunctionUniqCombinedBiasCorrectedData>(*argument_types[0]);
|
||||
|
||||
if (res)
|
||||
return res;
|
||||
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDate::FieldType,
|
||||
AggregateFunctionUniqCombinedBiasCorrectedData<DataTypeDate::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDateTime::FieldType,
|
||||
AggregateFunctionUniqCombinedBiasCorrectedData<DataTypeDateTime::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedBiasCorrectedData<String>>;
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "uniqCombined")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
@ -733,6 +799,9 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun
|
||||
"uniq",
|
||||
"uniqHLL12",
|
||||
"uniqExact",
|
||||
"uniqCombinedRaw",
|
||||
"uniqCombinedLinearCounting",
|
||||
"uniqCombinedBiasCorrected",
|
||||
"uniqCombined",
|
||||
"uniqUpTo",
|
||||
"groupArray",
|
||||
|
390
dbms/src/AggregateFunctions/UniqCombinedBiasData.cpp
Normal file
390
dbms/src/AggregateFunctions/UniqCombinedBiasData.cpp
Normal file
@ -0,0 +1,390 @@
|
||||
#include <DB/AggregateFunctions/UniqCombinedBiasData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
const UniqCombinedBiasData::InterpolatedData raw_estimates =
|
||||
{
|
||||
700.0
|
||||
,3850.0
|
||||
,7350.0
|
||||
,10850.0
|
||||
,14350.0
|
||||
,89003.5714
|
||||
,103764.30343333333
|
||||
,105572.1915
|
||||
,109252.46533333334
|
||||
,112638.20573333332
|
||||
,116094.29566666669
|
||||
,119619.81926666666
|
||||
,123214.92233333334
|
||||
,126469.06656666666
|
||||
,130196.15093333334
|
||||
,133566.85673333335
|
||||
,136991.63890000002
|
||||
,140470.0118666667
|
||||
,144000.91686666667
|
||||
,147585.44463333333
|
||||
,151222.7466
|
||||
,154447.75893333333
|
||||
,158181.68399999998
|
||||
,161492.41386666667
|
||||
,164840.6352
|
||||
,168713.9904
|
||||
,172143.82656666666
|
||||
,175611.2078
|
||||
,179116.94873333335
|
||||
,182658.0355
|
||||
,186236.36723333332
|
||||
,189332.1009
|
||||
,192976.1847
|
||||
,196654.62706666664
|
||||
,199835.39103333335
|
||||
,203575.92429999998
|
||||
,206808.87086666666
|
||||
,210611.72886666664
|
||||
,213896.25913333334
|
||||
,217759.63066666664
|
||||
,221096.10933333333
|
||||
,224456.31466666667
|
||||
,227839.0366333333
|
||||
,231242.72576666667
|
||||
,235239.98256666667
|
||||
,238688.95070000002
|
||||
,242158.17593333332
|
||||
,245649.42926666664
|
||||
,249158.9859666667
|
||||
,252689.67179999998
|
||||
,256241.95376666667
|
||||
,259214.9391666667
|
||||
,262798.3925666667
|
||||
,266399.8345666667
|
||||
,270018.35863333335
|
||||
,273653.1149
|
||||
,276696.7119
|
||||
,280366.51476666663
|
||||
,284051.95540000004
|
||||
,287133.5254333333
|
||||
,290847.31173333334
|
||||
,294579.5226
|
||||
,297698.64109999995
|
||||
,301454.39253333333
|
||||
,305223.59123333334
|
||||
,308375.3184666667
|
||||
,312170.06
|
||||
,315342.02996666665
|
||||
,319162.8188666667
|
||||
,322356.3565666666
|
||||
,326199.5866
|
||||
,329412.83396666666
|
||||
,332634.3235666667
|
||||
,336510.7596333333
|
||||
,339747.7330333333
|
||||
,343643.0385666667
|
||||
,346896.77420000004
|
||||
,350157.6729666667
|
||||
,354079.3932333334
|
||||
,357354.5196333334
|
||||
,360638.3034333333
|
||||
,364588.47873333335
|
||||
,367886.05706666666
|
||||
,371189.98006666667
|
||||
,375161.95876666665
|
||||
,378478.6737666666
|
||||
,381801.6619
|
||||
,385130.9645
|
||||
,389131.7460333333
|
||||
,392471.6233333333
|
||||
,395817.1175
|
||||
,399165.1003333333
|
||||
,402518.7819333333
|
||||
,406549.7624333333
|
||||
,409916.016
|
||||
,413289.0218666666
|
||||
,416661.9977333333
|
||||
,420040.4257333334
|
||||
,424099.3186333333
|
||||
,427485.4292000001
|
||||
,430876.4814666666
|
||||
,434269.4718
|
||||
,437665.82826666674
|
||||
,441066.7185
|
||||
,444469.97226666665
|
||||
,448561.9376666667
|
||||
,451974.73750000005
|
||||
,455389.1112
|
||||
,458808.5816666667
|
||||
,462230.8184666667
|
||||
,465656.9889
|
||||
,469081.3269
|
||||
,472512.4878
|
||||
,475944.4204333333
|
||||
,480065.7132666667
|
||||
,483502.04110000003
|
||||
,486939.5075666667
|
||||
,490379.7868333334
|
||||
,493818.5365333333
|
||||
,497259.08013333334
|
||||
,500705.3513
|
||||
,504155.6234666666
|
||||
,507606.65499999997
|
||||
,511060.7448666667
|
||||
,514517.4004
|
||||
,517973.35829999996
|
||||
,521431.3761666666
|
||||
,524891.7097333333
|
||||
,529044.7593
|
||||
,532507.0878999999
|
||||
,535971.5070333333
|
||||
,539436.2416999999
|
||||
,542903.1470333333
|
||||
,546370.3423
|
||||
,549837.6947999999
|
||||
,553307.0003666667
|
||||
,556775.3770333333
|
||||
,560247.6308333334
|
||||
,563721.0700333334
|
||||
,567196.7586333333
|
||||
,570669.8439666666
|
||||
,574146.018
|
||||
,577622.2794666667
|
||||
,581098.3862333334
|
||||
,584575.8826666666
|
||||
,588055.1468000001
|
||||
,591538.0234
|
||||
,595018.0103000001
|
||||
,598504.5469333333
|
||||
,601992.5697666666
|
||||
,605475.5452
|
||||
,608959.4645
|
||||
,612444.0261
|
||||
,615929.6436
|
||||
,619412.3877333334
|
||||
,622903.4263999999
|
||||
,626391.3657333333
|
||||
,629876.7359333333
|
||||
,633364.2825999999
|
||||
,636855.2673666667
|
||||
,640344.4321000001
|
||||
,643836.5543666667
|
||||
,647327.3073999999
|
||||
,650818.3525666667
|
||||
,654312.2421666667
|
||||
,657807.0899666668
|
||||
,661301.4443666666
|
||||
,664794.1040333334
|
||||
,668288.1969666666
|
||||
,671781.0196666667
|
||||
,675272.7522333333
|
||||
,678766.9045999999
|
||||
,682259.3583666667
|
||||
,685747.8148333334
|
||||
,689238.7994666666
|
||||
,692732.0478333334
|
||||
,696224.6407
|
||||
,700069.9224
|
||||
};
|
||||
|
||||
const UniqCombinedBiasData::InterpolatedData biases =
|
||||
{
|
||||
0.0
|
||||
,0.0
|
||||
,0.0
|
||||
,0.0
|
||||
,0.0
|
||||
,71153.5714
|
||||
,85214.30343333333
|
||||
,83522.1915
|
||||
,80202.46533333334
|
||||
,77288.20573333332
|
||||
,74444.29566666667
|
||||
,71669.81926666667
|
||||
,68964.92233333334
|
||||
,66619.06656666666
|
||||
,64046.15093333333
|
||||
,61816.85673333333
|
||||
,59641.6389
|
||||
,57520.01186666667
|
||||
,55450.91686666667
|
||||
,53435.44463333334
|
||||
,51472.74659999999
|
||||
,49797.75893333333
|
||||
,47931.68399999999
|
||||
,46342.41386666667
|
||||
,44790.6352
|
||||
,43063.9904
|
||||
,41593.82656666667
|
||||
,40161.2078
|
||||
,38766.94873333333
|
||||
,37408.035500000005
|
||||
,36086.36723333333
|
||||
,34982.1009
|
||||
,33726.184700000005
|
||||
,32504.627066666664
|
||||
,31485.391033333333
|
||||
,30325.924299999995
|
||||
,29358.870866666668
|
||||
,28261.72886666667
|
||||
,27346.259133333337
|
||||
,26309.630666666668
|
||||
,25446.109333333337
|
||||
,24606.31466666666
|
||||
,23789.036633333333
|
||||
,22992.725766666666
|
||||
,22089.98256666667
|
||||
,21338.9507
|
||||
,20608.175933333332
|
||||
,19899.429266666673
|
||||
,19208.985966666663
|
||||
,18539.6718
|
||||
,17891.95376666667
|
||||
,17364.939166666667
|
||||
,16748.392566666666
|
||||
,16149.834566666666
|
||||
,15568.358633333331
|
||||
,15003.114899999995
|
||||
,14546.711900000004
|
||||
,14016.51476666668
|
||||
,13501.955399999997
|
||||
,13083.52543333332
|
||||
,12597.311733333336
|
||||
,12129.522600000006
|
||||
,11748.641100000008
|
||||
,11304.392533333332
|
||||
,10873.59123333334
|
||||
,10525.318466666678
|
||||
,10120.059999999998
|
||||
,9792.029966666674
|
||||
,9412.818866666668
|
||||
,9106.356566666664
|
||||
,8749.58660000001
|
||||
,8462.833966666678
|
||||
,8184.323566666659
|
||||
,7860.759633333325
|
||||
,7597.733033333323
|
||||
,7293.038566666665
|
||||
,7046.774200000004
|
||||
,6807.672966666675
|
||||
,6529.393233333336
|
||||
,6304.519633333344
|
||||
,6088.30343333332
|
||||
,5838.4787333333325
|
||||
,5636.057066666661
|
||||
,5439.980066666671
|
||||
,5211.958766666658
|
||||
,5028.673766666664
|
||||
,4851.661899999996
|
||||
,4680.964499999992
|
||||
,4481.746033333319
|
||||
,4321.623333333322
|
||||
,4167.117500000012
|
||||
,4015.1003333333356
|
||||
,3868.781933333337
|
||||
,3699.762433333332
|
||||
,3566.0159999999937
|
||||
,3439.021866666648
|
||||
,3311.9977333333422
|
||||
,3190.4257333333276
|
||||
,3049.3186333333238
|
||||
,2935.4291999999937
|
||||
,2826.4814666666593
|
||||
,2719.4717999999993
|
||||
,2615.8282666666782
|
||||
,2516.7184999999977
|
||||
,2419.972266666669
|
||||
,2311.9376666666744
|
||||
,2224.7374999999884
|
||||
,2139.1111999999944
|
||||
,2058.581666666665
|
||||
,1980.8184666666687
|
||||
,1906.9888999999966
|
||||
,1831.3268999999952
|
||||
,1762.4878000000026
|
||||
,1694.420433333328
|
||||
,1615.7132666666682
|
||||
,1552.0410999999924
|
||||
,1489.507566666677
|
||||
,1429.7868333333365
|
||||
,1368.536533333332
|
||||
,1309.0801333333268
|
||||
,1255.35129999999
|
||||
,1205.6234666666617
|
||||
,1156.6549999999988
|
||||
,1110.744866666675
|
||||
,1067.4004000000034
|
||||
,1023.3583000000023
|
||||
,981.3761666666638
|
||||
,941.7097333333513
|
||||
,894.7593000000148
|
||||
,857.0879000000035
|
||||
,821.5070333333375
|
||||
,786.2416999999745
|
||||
,753.1470333333127
|
||||
,720.3422999999797
|
||||
,687.6947999999975
|
||||
,657.0003666666647
|
||||
,625.3770333333329
|
||||
,597.6308333333387
|
||||
,571.0700333333225
|
||||
,546.7586333333165
|
||||
,519.8439666666478
|
||||
,496.0180000000012
|
||||
,472.2794666666693
|
||||
,448.386233333343
|
||||
,425.8826666666816
|
||||
,405.1468000000071
|
||||
,388.0233999999861
|
||||
,368.01030000002356
|
||||
,354.54693333333125
|
||||
,342.5697666666626
|
||||
,325.5452000000126
|
||||
,309.4644999999825
|
||||
,294.0261000000173
|
||||
,279.64360000001034
|
||||
,262.38773333333666
|
||||
,253.42639999999665
|
||||
,241.36573333333945
|
||||
,226.7359333333443
|
||||
,214.28259999999622
|
||||
,205.26736666667662
|
||||
,194.43210000001514
|
||||
,186.55436666666841
|
||||
,177.30740000001
|
||||
,168.35256666666828
|
||||
,162.24216666668266
|
||||
,157.0899666666713
|
||||
,151.44436666666297
|
||||
,144.1040333333464
|
||||
,138.19696666668946
|
||||
,131.01966666666945
|
||||
,122.7522333333424
|
||||
,116.90459999998954
|
||||
,109.35836666667213
|
||||
,97.81483333332774
|
||||
,88.7994666666491
|
||||
,82.04783333333519
|
||||
,74.64070000000841
|
||||
,69.92240000003949
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
double UniqCombinedBiasData::getThreshold()
|
||||
{
|
||||
return 176000;
|
||||
}
|
||||
|
||||
const UniqCombinedBiasData::InterpolatedData & UniqCombinedBiasData::getRawEstimates()
|
||||
{
|
||||
return raw_estimates;
|
||||
}
|
||||
|
||||
const UniqCombinedBiasData::InterpolatedData & UniqCombinedBiasData::getBiases()
|
||||
{
|
||||
return biases;
|
||||
}
|
||||
|
||||
}
|
@ -177,7 +177,11 @@ Connection::Packet ParallelReplicas::drain()
|
||||
std::string ParallelReplicas::dumpAddresses() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
return dumpAddressesUnlocked();
|
||||
}
|
||||
|
||||
std::string ParallelReplicas::dumpAddressesUnlocked() const
|
||||
{
|
||||
bool is_first = true;
|
||||
std::ostringstream os;
|
||||
for (auto & e : replica_map)
|
||||
@ -289,13 +293,7 @@ ParallelReplicas::ReplicaMap::iterator ParallelReplicas::waitForReadEvent()
|
||||
int n = Poco::Net::Socket::select(read_list, write_list, except_list, settings->receive_timeout);
|
||||
|
||||
if (n == 0)
|
||||
{
|
||||
std::stringstream description;
|
||||
for (auto it = replica_map.begin(); it != replica_map.end(); ++it)
|
||||
description << (it != replica_map.begin() ? ", " : "") << it->second->getDescription();
|
||||
|
||||
throw Exception("Timeout exceeded while reading from " + description.str(), ErrorCodes::TIMEOUT_EXCEEDED);
|
||||
}
|
||||
throw Exception("Timeout exceeded while reading from " + dumpAddressesUnlocked(), ErrorCodes::TIMEOUT_EXCEEDED);
|
||||
}
|
||||
|
||||
auto & socket = read_list[rand() % read_list.size()];
|
||||
|
@ -53,7 +53,7 @@ void CreatingSetsBlockInputStream::create(SubqueryForSet & subquery)
|
||||
|
||||
BlockOutputStreamPtr table_out;
|
||||
if (subquery.table)
|
||||
table_out = subquery.table->write(ASTPtr());
|
||||
table_out = subquery.table->write({}, {});
|
||||
|
||||
bool done_with_set = !subquery.set;
|
||||
bool done_with_join = !subquery.join;
|
||||
|
@ -8,8 +8,8 @@ namespace DB
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
LimitBlockInputStream::LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_)
|
||||
: limit(limit_), offset(offset_), pos(0)
|
||||
LimitBlockInputStream::LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_)
|
||||
: limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_)
|
||||
{
|
||||
children.push_back(input_);
|
||||
}
|
||||
@ -23,7 +23,16 @@ Block LimitBlockInputStream::readImpl()
|
||||
/// pos - сколько строк было прочитано, включая последний прочитанный блок
|
||||
|
||||
if (pos >= offset + limit)
|
||||
return res;
|
||||
{
|
||||
if (!always_read_till_end)
|
||||
return res;
|
||||
else
|
||||
{
|
||||
while (children.back()->read())
|
||||
;
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
do
|
||||
{
|
||||
|
@ -0,0 +1,225 @@
|
||||
#include <DB/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
MergingAggregatedMemoryEfficientBlockInputStream::MergingAggregatedMemoryEfficientBlockInputStream(
|
||||
BlockInputStreams inputs_, const Names & keys_names_,
|
||||
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_)
|
||||
: aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0),
|
||||
final(final_),
|
||||
inputs(inputs_.begin(), inputs_.end())
|
||||
{
|
||||
children = inputs_;
|
||||
}
|
||||
|
||||
String MergingAggregatedMemoryEfficientBlockInputStream::getID() const
|
||||
{
|
||||
std::stringstream res;
|
||||
res << "MergingAggregatedMemoryEfficient(" << aggregator.getID();
|
||||
for (size_t i = 0, size = children.size(); i < size; ++i)
|
||||
res << ", " << children.back()->getID();
|
||||
res << ")";
|
||||
return res.str();
|
||||
}
|
||||
|
||||
Block MergingAggregatedMemoryEfficientBlockInputStream::readImpl()
|
||||
{
|
||||
/// Если child - RemoteBlockInputStream, то отправляет запрос на все удалённые серверы, инициируя вычисления.
|
||||
/** NOTE: Если соединения ещё не установлены, то устанавливает их последовательно.
|
||||
* И отправляет запрос последовательно. Это медленно.
|
||||
*/
|
||||
if (!started)
|
||||
{
|
||||
started = true;
|
||||
for (auto & child : children)
|
||||
child->readPrefix();
|
||||
}
|
||||
|
||||
/** Имеем несколько источников.
|
||||
* Из каждого из них могут приходить следующие данные:
|
||||
*
|
||||
* 1. Блок, с указанным bucket_num.
|
||||
* Это значит, что на удалённом сервере, данные были разрезаны по корзинам.
|
||||
* И данные для одного bucket_num с разных серверов можно независимо объединять.
|
||||
* При этом, даннные для разных bucket_num будут идти по возрастанию.
|
||||
*
|
||||
* 2. Блок без указания bucket_num.
|
||||
* Это значит, что на удалённом сервере, данные не были разрезаны по корзинам.
|
||||
* В случае, когда со всех серверов прийдут такие данные, их можно всех объединить.
|
||||
* А если с другой части серверов прийдут данные, разрезанные по корзинам,
|
||||
* то данные, не разрезанные по корзинам, нужно сначала разрезать, а потом объединять.
|
||||
*
|
||||
* 3. Блоки с указанием is_overflows.
|
||||
* Это дополнительные данные для строк, не прошедших через max_rows_to_group_by.
|
||||
* Они должны объединяться друг с другом отдельно.
|
||||
*/
|
||||
|
||||
constexpr size_t NUM_BUCKETS = 256;
|
||||
|
||||
++current_bucket_num;
|
||||
|
||||
for (auto & input : inputs)
|
||||
{
|
||||
if (input.is_exhausted)
|
||||
continue;
|
||||
|
||||
if (input.block.info.bucket_num >= current_bucket_num)
|
||||
continue;
|
||||
|
||||
/// Если придёт блок не с основными данными, а с overflows, то запомним его и повторим чтение.
|
||||
while (true)
|
||||
{
|
||||
// std::cerr << "reading block\n";
|
||||
Block block = input.stream->read();
|
||||
|
||||
if (!block)
|
||||
{
|
||||
// std::cerr << "input is exhausted\n";
|
||||
input.is_exhausted = true;
|
||||
break;
|
||||
}
|
||||
|
||||
if (block.info.bucket_num != -1)
|
||||
{
|
||||
/// Один из разрезанных блоков для двухуровневых данных.
|
||||
// std::cerr << "block for bucket " << block.info.bucket_num << "\n";
|
||||
|
||||
has_two_level = true;
|
||||
input.block = block;
|
||||
}
|
||||
else if (block.info.is_overflows)
|
||||
{
|
||||
// std::cerr << "block for overflows\n";
|
||||
|
||||
has_overflows = true;
|
||||
input.overflow_block = block;
|
||||
|
||||
continue;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Блок для неразрезанных (одноуровневых) данных.
|
||||
// std::cerr << "block without bucket\n";
|
||||
|
||||
input.block = block;
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
while (true)
|
||||
{
|
||||
if (current_bucket_num == NUM_BUCKETS)
|
||||
{
|
||||
/// Обработали все основные данные. Остались, возможно, только overflows-блоки.
|
||||
// std::cerr << "at end\n";
|
||||
|
||||
if (has_overflows)
|
||||
{
|
||||
// std::cerr << "merging overflows\n";
|
||||
|
||||
has_overflows = false;
|
||||
BlocksList blocks_to_merge;
|
||||
|
||||
for (auto & input : inputs)
|
||||
if (input.overflow_block)
|
||||
blocks_to_merge.emplace_back(std::move(input.overflow_block));
|
||||
|
||||
return aggregator.mergeBlocks(blocks_to_merge, final);
|
||||
}
|
||||
else
|
||||
return {};
|
||||
}
|
||||
else if (has_two_level)
|
||||
{
|
||||
/** Есть двухуровневые данные.
|
||||
* Будем обрабатывать номера корзин по возрастанию.
|
||||
* Найдём минимальный номер корзины, для которой есть данные,
|
||||
* затем померджим эти данные.
|
||||
*/
|
||||
// std::cerr << "has two level\n";
|
||||
|
||||
int min_bucket_num = NUM_BUCKETS;
|
||||
|
||||
for (auto & input : inputs)
|
||||
{
|
||||
/// Изначально разрезанные (двухуровневые) блоки.
|
||||
if (input.block.info.bucket_num != -1 && input.block.info.bucket_num < min_bucket_num)
|
||||
min_bucket_num = input.block.info.bucket_num;
|
||||
|
||||
/// Ещё не разрезанный по корзинам блок. Разрезаем его и кладём результат в splitted_blocks.
|
||||
if (input.block.info.bucket_num == -1 && input.block && input.splitted_blocks.empty())
|
||||
{
|
||||
LOG_TRACE(&Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split.");
|
||||
|
||||
input.splitted_blocks = aggregator.convertBlockToTwoLevel(input.block);
|
||||
input.block = Block();
|
||||
}
|
||||
|
||||
/// Блоки, которые мы получили разрезанием одноуровневых блоков.
|
||||
if (!input.splitted_blocks.empty())
|
||||
{
|
||||
for (const auto & block : input.splitted_blocks)
|
||||
{
|
||||
if (block && block.info.bucket_num < min_bucket_num)
|
||||
{
|
||||
min_bucket_num = block.info.bucket_num;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
current_bucket_num = min_bucket_num;
|
||||
|
||||
// std::cerr << "current_bucket_num = " << current_bucket_num << "\n";
|
||||
|
||||
/// Блоков с основными данными больше нет.
|
||||
if (current_bucket_num == NUM_BUCKETS)
|
||||
continue;
|
||||
|
||||
/// Теперь собираем блоки для current_bucket_num, чтобы их померджить.
|
||||
BlocksList blocks_to_merge;
|
||||
|
||||
for (auto & input : inputs)
|
||||
{
|
||||
if (input.block.info.bucket_num == current_bucket_num)
|
||||
{
|
||||
// std::cerr << "having block for current_bucket_num\n";
|
||||
|
||||
blocks_to_merge.emplace_back(std::move(input.block));
|
||||
input.block = Block();
|
||||
}
|
||||
else if (!input.splitted_blocks.empty() && input.splitted_blocks[min_bucket_num])
|
||||
{
|
||||
// std::cerr << "having splitted data for bucket\n";
|
||||
|
||||
blocks_to_merge.emplace_back(std::move(input.splitted_blocks[min_bucket_num]));
|
||||
input.splitted_blocks[min_bucket_num] = Block();
|
||||
}
|
||||
}
|
||||
|
||||
return aggregator.mergeBlocks(blocks_to_merge, final);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Есть только одноуровневые данные. Просто мерджим их.
|
||||
// std::cerr << "don't have two level\n";
|
||||
|
||||
BlocksList blocks_to_merge;
|
||||
|
||||
for (auto & input : inputs)
|
||||
if (input.block)
|
||||
blocks_to_merge.emplace_back(std::move(input.block));
|
||||
|
||||
current_bucket_num = NUM_BUCKETS;
|
||||
return aggregator.mergeBlocks(blocks_to_merge, final);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -132,7 +132,7 @@ int main(int argc, char ** argv)
|
||||
Poco::SharedPtr<IBlockInputStream> in = table->read(column_names, 0, context, Settings(), stage)[0];
|
||||
in = new ExpressionBlockInputStream(in, expression);
|
||||
in = new FilterBlockInputStream(in, 4);
|
||||
//in = new LimitBlockInputStream(in, 10);
|
||||
//in = new LimitBlockInputStream(in, 10, 0);
|
||||
|
||||
WriteBufferFromOStream ob(std::cout);
|
||||
RowOutputStreamPtr out_ = new TabSeparatedRowOutputStream(ob, expression->getSampleBlock());
|
||||
|
@ -119,7 +119,7 @@ int main(int argc, char ** argv)
|
||||
ReadBufferFromFileDescriptor in1(STDIN_FILENO);
|
||||
CompressedReadBuffer in2(in1);
|
||||
NativeBlockInputStream in3(in2, Revision::get());
|
||||
SharedPtr<IBlockOutputStream> out = table->write(0);
|
||||
SharedPtr<IBlockOutputStream> out = table->write({}, {});
|
||||
copyData(in3, *out);
|
||||
}
|
||||
}
|
||||
|
@ -152,7 +152,7 @@ int main(int argc, char ** argv)
|
||||
Poco::SharedPtr<IBlockInputStream> in = table->read(column_names, 0, Context{}, Settings(), stage, argc == 2 ? atoi(argv[1]) : 1048576)[0];
|
||||
in = new PartialSortingBlockInputStream(in, sort_columns);
|
||||
in = new MergeSortingBlockInputStream(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, "");
|
||||
//in = new LimitBlockInputStream(in, 10);
|
||||
//in = new LimitBlockInputStream(in, 10, 0);
|
||||
|
||||
WriteBufferFromOStream ob(std::cout);
|
||||
RowOutputStreamPtr out_ = new TabSeparatedRowOutputStream(ob, sample);
|
||||
|
@ -45,7 +45,7 @@ int main(int argc, char ** argv)
|
||||
streams[i] = new DB::AsynchronousBlockInputStream(streams[i]);
|
||||
|
||||
DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, nullptr, settings.max_threads);
|
||||
stream = new DB::LimitBlockInputStream(stream, 10);
|
||||
stream = new DB::LimitBlockInputStream(stream, 10, 0);
|
||||
|
||||
DB::FormatFactory format_factory;
|
||||
DB::WriteBufferFromFileDescriptor wb(STDERR_FILENO);
|
||||
|
@ -8,10 +8,12 @@ void registerFunctionsCoding(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionToStringCutToZero>();
|
||||
factory.registerFunction<FunctionIPv6NumToString>();
|
||||
factory.registerFunction<FunctionCutIPv6>();
|
||||
factory.registerFunction<FunctionIPv6StringToNum>();
|
||||
factory.registerFunction<FunctionIPv4NumToString>();
|
||||
factory.registerFunction<FunctionIPv4StringToNum>();
|
||||
factory.registerFunction<FunctionIPv4NumToStringClassC>();
|
||||
factory.registerFunction<FunctionIPv4ToIPv6>();
|
||||
factory.registerFunction<FunctionHex>();
|
||||
factory.registerFunction<FunctionUnhex>();
|
||||
factory.registerFunction<FunctionBitmaskToArray>();
|
||||
|
@ -8,6 +8,7 @@ void registerFunctionsRandom(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionRand>();
|
||||
factory.registerFunction<FunctionRand64>();
|
||||
factory.registerFunction<FunctionRandConstant>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -11,11 +11,10 @@ namespace DB
|
||||
|
||||
/// Примечание: выделяется дополнительная страница, которая содежрит те данные, которые
|
||||
/// не влезают в основной буфер.
|
||||
ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_,
|
||||
char * existing_memory_)
|
||||
ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, char * existing_memory_)
|
||||
: ReadBufferFromFileBase(buffer_size_ + DEFAULT_AIO_FILE_BLOCK_SIZE, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE),
|
||||
fill_buffer(BufferWithOwnMemory<ReadBuffer>(this->memory.size(), nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)),
|
||||
filename(filename_)
|
||||
fill_buffer(BufferWithOwnMemory<ReadBuffer>(internalBuffer().size(), nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)),
|
||||
filename(filename_)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::FileOpen);
|
||||
|
||||
|
@ -58,7 +58,7 @@ void AggregatedDataVariants::convertToTwoLevel()
|
||||
}
|
||||
|
||||
|
||||
void Aggregator::initialize(Block & block)
|
||||
void Aggregator::initialize(const Block & block)
|
||||
{
|
||||
if (isCancelled())
|
||||
return;
|
||||
@ -1699,7 +1699,11 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
|
||||
AggregateColumnsData aggregate_columns(aggregates_size);
|
||||
|
||||
initialize(blocks.front());
|
||||
Block empty_block;
|
||||
initialize(empty_block);
|
||||
|
||||
if (!sample)
|
||||
sample = blocks.front().cloneEmpty();
|
||||
|
||||
/// Каким способом выполнять агрегацию?
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
@ -1735,17 +1739,190 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
}
|
||||
|
||||
BlocksList merged_block = convertToBlocks(result, final, 1);
|
||||
BlocksList merged_blocks = convertToBlocks(result, final, 1);
|
||||
|
||||
if (merged_block.size() > 1) /// TODO overflows
|
||||
throw Exception("Logical error: temporary result is not single-level", ErrorCodes::LOGICAL_ERROR);
|
||||
if (merged_blocks.size() > 1)
|
||||
{
|
||||
/** Может быть два блока. Один с is_overflows, другой - нет.
|
||||
* Если есть непустой блок не is_overflows, то удаляем блок с is_overflows.
|
||||
* Если есть пустой блок не is_overflows и блок с is_overflows, то удаляем пустой блок.
|
||||
*
|
||||
* Это делаем, потому что исходим из допущения, что в функцию передаются
|
||||
* либо все блоки не is_overflows, либо все блоки is_overflows.
|
||||
*/
|
||||
|
||||
bool has_nonempty_nonoverflows = false;
|
||||
bool has_overflows = false;
|
||||
|
||||
for (const auto & block : merged_blocks)
|
||||
{
|
||||
if (block && block.rowsInFirstColumn() && !block.info.is_overflows)
|
||||
has_nonempty_nonoverflows = true;
|
||||
else if (block.info.is_overflows)
|
||||
has_overflows = true;
|
||||
}
|
||||
|
||||
if (has_nonempty_nonoverflows)
|
||||
{
|
||||
for (auto it = merged_blocks.begin(); it != merged_blocks.end(); ++it)
|
||||
{
|
||||
if (it->info.is_overflows)
|
||||
{
|
||||
merged_blocks.erase(it);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (has_overflows)
|
||||
{
|
||||
for (auto it = merged_blocks.begin(); it != merged_blocks.end(); ++it)
|
||||
{
|
||||
if (!*it || it->rowsInFirstColumn() == 0)
|
||||
{
|
||||
merged_blocks.erase(it);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (merged_blocks.size() > 1)
|
||||
throw Exception("Logical error: temporary result is not single-level", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Merged partially aggregated blocks.");
|
||||
|
||||
if (merged_block.empty())
|
||||
if (merged_blocks.empty())
|
||||
return {};
|
||||
|
||||
return merged_block.front();
|
||||
return merged_blocks.front();
|
||||
}
|
||||
|
||||
|
||||
template <typename Method>
|
||||
void NO_INLINE Aggregator::convertBlockToTwoLevelImpl(
|
||||
Method & method,
|
||||
Arena * pool,
|
||||
ConstColumnPlainPtrs & key_columns,
|
||||
const Sizes & key_sizes,
|
||||
StringRefs & keys,
|
||||
const Block & source,
|
||||
std::vector<Block> & destinations) const
|
||||
{
|
||||
typename Method::State state;
|
||||
state.init(key_columns);
|
||||
|
||||
size_t rows = source.rowsInFirstColumn();
|
||||
size_t columns = source.columns();
|
||||
|
||||
/// Для каждого номера корзины создадим фильтр, где будут отмечены строки, относящиеся к этой корзине.
|
||||
std::vector<IColumn::Filter> filters(destinations.size());
|
||||
|
||||
/// Для всех строчек.
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
/// Получаем ключ. Вычисляем на его основе номер корзины.
|
||||
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes, keys, *pool);
|
||||
|
||||
auto hash = method.data.hash(key);
|
||||
auto bucket = method.data.getBucketFromHash(hash);
|
||||
|
||||
/// Этот ключ нам больше не нужен.
|
||||
method.onExistingKey(key, keys, *pool);
|
||||
|
||||
auto & filter = filters[bucket];
|
||||
|
||||
if (unlikely(filter.empty()))
|
||||
filter.resize_fill(rows);
|
||||
|
||||
filter[i] = 1;
|
||||
}
|
||||
|
||||
for (size_t bucket = 0, size = destinations.size(); bucket < size; ++bucket)
|
||||
{
|
||||
const auto & filter = filters[bucket];
|
||||
|
||||
if (filter.empty())
|
||||
continue;
|
||||
|
||||
Block & dst = destinations[bucket];
|
||||
dst.info.bucket_num = bucket;
|
||||
|
||||
for (size_t j = 0; j < columns; ++j)
|
||||
{
|
||||
const ColumnWithTypeAndName & src_col = source.unsafeGetByPosition(j);
|
||||
dst.insert({src_col.column->filter(filter), src_col.type, src_col.name});
|
||||
|
||||
/** Вставленные в блок столбцы типа ColumnAggregateFunction будут владеть состояниями агрегатных функций
|
||||
* путём удержания SharedPtr-а на исходный столбец. См. ColumnAggregateFunction.h
|
||||
*/
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
|
||||
{
|
||||
if (!block)
|
||||
return {};
|
||||
|
||||
Block empty_block;
|
||||
initialize(empty_block);
|
||||
|
||||
if (!sample)
|
||||
sample = block.cloneEmpty();
|
||||
|
||||
AggregatedDataVariants data;
|
||||
|
||||
StringRefs key(keys_size);
|
||||
ConstColumnPlainPtrs key_columns(keys_size);
|
||||
Sizes key_sizes;
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
key_columns[i] = block.getByPosition(i).column;
|
||||
|
||||
AggregatedDataVariants::Type type = chooseAggregationMethod(key_columns, key_sizes);
|
||||
data.keys_size = keys_size;
|
||||
data.key_sizes = key_sizes;
|
||||
|
||||
#define M(NAME) \
|
||||
else if (type == AggregatedDataVariants::Type::NAME) \
|
||||
type = AggregatedDataVariants::Type::NAME ## _two_level;
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
|
||||
#undef M
|
||||
else
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
|
||||
data.init(type);
|
||||
|
||||
size_t num_buckets = 0;
|
||||
|
||||
#define M(NAME) \
|
||||
else if (data.type == AggregatedDataVariants::Type::NAME) \
|
||||
num_buckets = data.NAME->data.NUM_BUCKETS;
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
||||
#undef M
|
||||
else
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
|
||||
std::vector<Block> splitted_blocks(num_buckets);
|
||||
|
||||
#define M(NAME) \
|
||||
else if (data.type == AggregatedDataVariants::Type::NAME) \
|
||||
convertBlockToTwoLevelImpl(*data.NAME, data.aggregates_pool, \
|
||||
key_columns, data.key_sizes, key, block, splitted_blocks);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
||||
#undef M
|
||||
else
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
|
||||
return splitted_blocks;
|
||||
}
|
||||
|
||||
|
||||
|
@ -366,55 +366,68 @@ StoragePtr Context::tryGetExternalTable(const String & table_name) const
|
||||
|
||||
StoragePtr Context::getTable(const String & database_name, const String & table_name) const
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||
|
||||
if (database_name.empty())
|
||||
{
|
||||
StoragePtr res;
|
||||
if ((res = tryGetExternalTable(table_name)))
|
||||
return res;
|
||||
if (session_context && (res = session_context->tryGetExternalTable(table_name)))
|
||||
return res;
|
||||
if (global_context && (res = global_context->tryGetExternalTable(table_name)))
|
||||
return res;
|
||||
}
|
||||
String db = database_name.empty() ? current_database : database_name;
|
||||
|
||||
Databases::const_iterator it = shared->databases.find(db);
|
||||
if (shared->databases.end() == it)
|
||||
throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
|
||||
|
||||
Tables::const_iterator jt = it->second.find(table_name);
|
||||
if (it->second.end() == jt)
|
||||
throw Exception("Table " + db + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
return jt->second;
|
||||
Exception exc;
|
||||
auto res = getTableImpl(database_name, table_name, &exc);
|
||||
if (!res)
|
||||
throw exc;
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
StoragePtr Context::tryGetTable(const String & database_name, const String & table_name) const
|
||||
{
|
||||
return getTableImpl(database_name, table_name, nullptr);
|
||||
}
|
||||
|
||||
|
||||
StoragePtr Context::getTableImpl(const String & database_name, const String & table_name, Exception * exception) const
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||
|
||||
/** Возможность обратиться к временным таблицам другого запроса в виде _query_QUERY_ID.table
|
||||
* NOTE В дальнейшем может потребоваться подумать об изоляции.
|
||||
*/
|
||||
if (database_name.size() > strlen("_query_")
|
||||
&& database_name.compare(0, strlen("_query_"), "_query_") == 0)
|
||||
{
|
||||
String requested_query_id = database_name.substr(strlen("_query_"));
|
||||
|
||||
auto res = shared->process_list.tryGetTemporaryTable(requested_query_id, table_name);
|
||||
|
||||
if (!res && exception)
|
||||
*exception = Exception(
|
||||
"Cannot find temporary table with name " + table_name + " for query with id " + requested_query_id, ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
if (database_name.empty())
|
||||
{
|
||||
StoragePtr res;
|
||||
if ((res = tryGetExternalTable(table_name)))
|
||||
return res;
|
||||
if (session_context && (res = session_context->tryGetExternalTable(table_name)))
|
||||
return res;
|
||||
if (global_context && (res = global_context->tryGetExternalTable(table_name)))
|
||||
StoragePtr res = tryGetExternalTable(table_name);
|
||||
if (res)
|
||||
return res;
|
||||
}
|
||||
|
||||
String db = database_name.empty() ? current_database : database_name;
|
||||
|
||||
Databases::const_iterator it = shared->databases.find(db);
|
||||
if (shared->databases.end() == it)
|
||||
return StoragePtr();
|
||||
{
|
||||
if (exception)
|
||||
*exception = Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
|
||||
return {};
|
||||
}
|
||||
|
||||
Tables::const_iterator jt = it->second.find(table_name);
|
||||
if (it->second.end() == jt)
|
||||
return StoragePtr();
|
||||
{
|
||||
if (exception)
|
||||
*exception = Exception("Table " + db + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
return {};
|
||||
}
|
||||
|
||||
if (!jt->second)
|
||||
throw Exception("Logical error: entry for table " + db + "." + table_name + " exists in Context but it is nullptr.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return jt->second;
|
||||
}
|
||||
@ -424,7 +437,14 @@ void Context::addExternalTable(const String & table_name, StoragePtr storage)
|
||||
{
|
||||
if (external_tables.end() != external_tables.find(table_name))
|
||||
throw Exception("Temporary table " + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
|
||||
external_tables[table_name] = storage;
|
||||
|
||||
if (process_list_elem)
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||
shared->process_list.addTemporaryTable(*process_list_elem, table_name, storage);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -590,6 +610,9 @@ void Context::setCurrentDatabase(const String & name)
|
||||
|
||||
void Context::setCurrentQueryId(const String & query_id)
|
||||
{
|
||||
if (!current_query_id.empty())
|
||||
throw Exception("Logical error: attempt to set query_id twice", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String query_id_to_set = query_id;
|
||||
if (query_id_to_set.empty()) /// Если пользователь не передал свой query_id, то генерируем его самостоятельно.
|
||||
query_id_to_set = shared->uuid_generator.createRandom().toString();
|
||||
|
@ -37,7 +37,7 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
|
||||
|
||||
if ("range_hashed" == layout_type)
|
||||
{
|
||||
if (dict_struct.range_min.empty() || dict_struct.range_min.empty())
|
||||
if (!dict_struct.range_min || !dict_struct.range_min)
|
||||
throw Exception{
|
||||
name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max",
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
@ -47,7 +47,7 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!dict_struct.range_min.empty() || !dict_struct.range_min.empty())
|
||||
if (dict_struct.range_min || dict_struct.range_min)
|
||||
throw Exception{
|
||||
name + ": elements .structure.range_min and .structure.range_max should be defined only "
|
||||
"for a dictionary of layout 'range_hashed'",
|
||||
|
@ -1,3 +1,5 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
#include <DB/DataTypes/FieldToDataType.h>
|
||||
|
||||
#include <DB/Parsers/ASTFunction.h>
|
||||
@ -20,6 +22,7 @@
|
||||
|
||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Interpreters/InJoinSubqueriesPreprocessor.h>
|
||||
#include <DB/Interpreters/LogicalExpressionsOptimizer.h>
|
||||
#include <DB/Interpreters/ExternalDictionaries.h>
|
||||
|
||||
@ -87,25 +90,31 @@ const std::unordered_set<String> possibly_injective_function_names
|
||||
"dictGetDateTime"
|
||||
};
|
||||
|
||||
static bool functionIsInOperator(const String & name)
|
||||
namespace
|
||||
{
|
||||
|
||||
bool functionIsInOperator(const String & name)
|
||||
{
|
||||
return name == "in" || name == "notIn";
|
||||
}
|
||||
|
||||
static bool functionIsInOrGlobalInOperator(const String & name)
|
||||
bool functionIsInOrGlobalInOperator(const String & name)
|
||||
{
|
||||
return name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn";
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::init()
|
||||
{
|
||||
select_query = typeid_cast<ASTSelectQuery *>(&*ast);
|
||||
|
||||
/// В зависимости от профиля пользователя проверить наличие прав на выполнение
|
||||
/// распределённых подзапросов внутри секций IN или JOIN и обработать эти подзапросы.
|
||||
InJoinSubqueriesPreprocessor<>(select_query, context, storage).perform();
|
||||
|
||||
/// Оптимизирует логические выражения.
|
||||
LogicalExpressionsOptimizer logical_expressions_optimizer(select_query, settings);
|
||||
logical_expressions_optimizer.optimizeDisjunctiveEqualityChains();
|
||||
LogicalExpressionsOptimizer(select_query, settings).optimizeDisjunctiveEqualityChains();
|
||||
|
||||
/// Добавляет в множество известных алиасов те, которые объявлены в структуре таблицы (ALIAS-столбцы).
|
||||
addStorageAliases();
|
||||
@ -886,8 +895,45 @@ static SharedPtr<InterpreterSelectQuery> interpretSubquery(
|
||||
select_query->children.emplace_back(select_query->table);
|
||||
}
|
||||
else
|
||||
{
|
||||
query = subquery->children.at(0);
|
||||
|
||||
/** В подзапросе могут быть указаны столбцы с одинаковыми именами. Например, SELECT x, x FROM t
|
||||
* Это плохо, потому что результат такого запроса нельзя сохранить в таблицу, потому что в таблице не может быть одноимённых столбцов.
|
||||
* Сохранение в таблицу требуется для GLOBAL-подзапросов.
|
||||
*
|
||||
* Чтобы избежать такой ситуации, будем переименовывать одинаковые столбцы.
|
||||
*/
|
||||
|
||||
std::set<std::string> all_column_names;
|
||||
std::set<std::string> assigned_column_names;
|
||||
|
||||
if (ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(query.get()))
|
||||
{
|
||||
for (auto & expr : select->select_expression_list->children)
|
||||
all_column_names.insert(expr->getAliasOrColumnName());
|
||||
|
||||
for (auto & expr : select->select_expression_list->children)
|
||||
{
|
||||
auto name = expr->getAliasOrColumnName();
|
||||
|
||||
if (!assigned_column_names.insert(name).second)
|
||||
{
|
||||
size_t i = 1;
|
||||
while (all_column_names.end() != all_column_names.find(name + "_" + toString(i)))
|
||||
++i;
|
||||
|
||||
name = name + "_" + toString(i);
|
||||
expr = expr->clone(); /// Отменяет склейку одинаковых выражений в дереве.
|
||||
expr->setAlias(name);
|
||||
|
||||
all_column_names.insert(name);
|
||||
assigned_column_names.insert(name);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (required_columns.empty())
|
||||
return new InterpreterSelectQuery(query, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1);
|
||||
else
|
||||
@ -897,6 +943,10 @@ static SharedPtr<InterpreterSelectQuery> interpretSubquery(
|
||||
|
||||
void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
|
||||
{
|
||||
/// При нераспределённых запросах, создание временных таблиц не имеет смысла.
|
||||
if (!(storage && storage->isRemote()))
|
||||
return;
|
||||
|
||||
if (const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(&*subquery_or_table_name))
|
||||
{
|
||||
/// Если это уже внешняя таблица, ничего заполять не нужно. Просто запоминаем ее наличие.
|
||||
@ -921,15 +971,73 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
|
||||
Block sample = interpreter->getSampleBlock();
|
||||
NamesAndTypesListPtr columns = new NamesAndTypesList(sample.getColumnsList());
|
||||
|
||||
/** Заменяем подзапрос на имя временной таблицы.
|
||||
* Именно в таком виде, запрос отправится на удалённый сервер.
|
||||
* На удалённый сервер отправится эта временная таблица, и на его стороне,
|
||||
* вместо выполнения подзапроса, надо будет просто из неё прочитать.
|
||||
*/
|
||||
subquery_or_table_name = new ASTIdentifier(StringRange(), external_table_name, ASTIdentifier::Table);
|
||||
|
||||
StoragePtr external_storage = StorageMemory::create(external_table_name, columns);
|
||||
|
||||
/** Есть два способа выполнения распределённых GLOBAL-подзапросов.
|
||||
*
|
||||
* Способ push:
|
||||
* Данные подзапроса отправляются на все удалённые серверы, где они затем используются.
|
||||
* Для этого способа, данные отправляются в виде "внешних таблиц" и будут доступны на каждом удалённом сервере по имени типа _data1.
|
||||
* Заменяем в запросе подзапрос на это имя.
|
||||
*
|
||||
* Способ pull:
|
||||
* Удалённые серверы скачивают данные подзапроса с сервера-инициатора запроса.
|
||||
* Для этого способа, заменяем подзапрос на другой подзапрос вида (SELECT * FROM remote('host:port', _query_QUERY_ID, _data1))
|
||||
* Этот подзапрос, по факту, говорит - "надо скачать данные оттуда".
|
||||
*
|
||||
* Способ pull имеет преимущество, потому что в нём удалённый сервер может решить, что ему не нужны данные и не скачивать их в таких случаях.
|
||||
*/
|
||||
|
||||
if (settings.global_subqueries_method == GlobalSubqueriesMethod::PUSH)
|
||||
{
|
||||
/** Заменяем подзапрос на имя временной таблицы.
|
||||
* Именно в таком виде, запрос отправится на удалённый сервер.
|
||||
* На удалённый сервер отправится эта временная таблица, и на его стороне,
|
||||
* вместо выполнения подзапроса, надо будет просто из неё прочитать.
|
||||
*/
|
||||
|
||||
subquery_or_table_name = new ASTIdentifier(StringRange(), external_table_name, ASTIdentifier::Table);
|
||||
}
|
||||
else if (settings.global_subqueries_method == GlobalSubqueriesMethod::PULL)
|
||||
{
|
||||
String host_port = getFQDNOrHostName() + ":" + Poco::Util::Application::instance().config().getString("tcp_port");
|
||||
String database = "_query_" + context.getCurrentQueryId();
|
||||
|
||||
auto subquery = new ASTSubquery;
|
||||
subquery_or_table_name = subquery;
|
||||
|
||||
auto select = new ASTSelectQuery;
|
||||
subquery->children.push_back(select);
|
||||
|
||||
auto exp_list = new ASTExpressionList;
|
||||
select->select_expression_list = exp_list;
|
||||
select->children.push_back(select->select_expression_list);
|
||||
|
||||
Names column_names = external_storage->getColumnNamesList();
|
||||
for (const auto & name : column_names)
|
||||
exp_list->children.push_back(new ASTIdentifier({}, name));
|
||||
|
||||
auto table_func = new ASTFunction;
|
||||
select->table = table_func;
|
||||
select->children.push_back(select->table);
|
||||
|
||||
table_func->name = "remote";
|
||||
auto args = new ASTExpressionList;
|
||||
table_func->arguments = args;
|
||||
table_func->children.push_back(table_func->arguments);
|
||||
|
||||
auto address_lit = new ASTLiteral({}, host_port);
|
||||
args->children.push_back(address_lit);
|
||||
|
||||
auto database_lit = new ASTLiteral({}, database);
|
||||
args->children.push_back(database_lit);
|
||||
|
||||
auto table_lit = new ASTLiteral({}, external_table_name);
|
||||
args->children.push_back(table_lit);
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown global subqueries execution method", ErrorCodes::UNKNOWN_GLOBAL_SUBQUERIES_METHOD);
|
||||
|
||||
external_tables[external_table_name] = external_storage;
|
||||
subqueries_for_sets[external_table_name].source = interpreter->execute().in;
|
||||
subqueries_for_sets[external_table_name].source_sample = interpreter->getSampleBlock();
|
||||
@ -1050,6 +1158,10 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
|
||||
void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sample_block, bool create_ordered_set)
|
||||
{
|
||||
IAST & args = *node->arguments;
|
||||
|
||||
if (args.children.size() != 2)
|
||||
throw Exception("Wrong number of arguments passed to function in", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTPtr & arg = args.children.at(1);
|
||||
|
||||
DataTypes set_element_types;
|
||||
@ -1087,12 +1199,20 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl
|
||||
{
|
||||
if (set_func->name == "tuple")
|
||||
{
|
||||
/// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2).
|
||||
ASTFunction * any_element = typeid_cast<ASTFunction *>(&*set_func->arguments->children.at(0));
|
||||
if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple"))
|
||||
single_value = true;
|
||||
else
|
||||
if (set_func->arguments->children.empty())
|
||||
{
|
||||
/// Пустое множество.
|
||||
elements_ast = set_func->arguments;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Отличм случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2).
|
||||
ASTFunction * any_element = typeid_cast<ASTFunction *>(&*set_func->arguments->children.at(0));
|
||||
if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple"))
|
||||
single_value = true;
|
||||
else
|
||||
elements_ast = set_func->arguments;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1743,15 +1863,13 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
|
||||
if (!subquery_for_set.join)
|
||||
{
|
||||
Names join_key_names_left(join_key_names_left_set.begin(), join_key_names_left_set.end());
|
||||
Names join_key_names_right(join_key_names_right_set.begin(), join_key_names_right_set.end());
|
||||
JoinPtr join = new Join(join_key_names_left, join_key_names_right, settings.limits, ast_join.kind, ast_join.strictness);
|
||||
|
||||
Names required_joined_columns(join_key_names_right.begin(), join_key_names_right.end());
|
||||
for (const auto & name_type : columns_added_by_join)
|
||||
required_joined_columns.push_back(name_type.name);
|
||||
|
||||
/** Для GLOBAL JOIN-ов происходит следующее:
|
||||
/** Для GLOBAL JOIN-ов (в случае, например, push-метода выполнения GLOBAL подзапросов) происходит следующее:
|
||||
* - в функции addExternalStorage подзапрос JOIN (SELECT ...) заменяется на JOIN _data1,
|
||||
* в объекте subquery_for_set выставляется этот подзапрос в качестве source и временная таблица _data1 в качестве table.
|
||||
* - в этой функции видно выражение JOIN _data1.
|
||||
@ -2137,27 +2255,32 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
|
||||
auto & keys = typeid_cast<ASTExpressionList &>(*node.using_expr_list);
|
||||
for (const auto & key : keys.children)
|
||||
{
|
||||
if (!join_key_names_left_set.insert(key->getColumnName()).second)
|
||||
throw Exception("Duplicate column in USING list", ErrorCodes::DUPLICATE_COLUMN);
|
||||
if (join_key_names_left.end() == std::find(join_key_names_left.begin(), join_key_names_left.end(), key->getColumnName()))
|
||||
join_key_names_left.push_back(key->getColumnName());
|
||||
else
|
||||
throw Exception("Duplicate column " + key->getColumnName() + " in USING list", ErrorCodes::DUPLICATE_COLUMN);
|
||||
|
||||
if (!join_key_names_right_set.insert(key->getAliasOrColumnName()).second)
|
||||
throw Exception("Duplicate column in USING list", ErrorCodes::DUPLICATE_COLUMN);
|
||||
if (join_key_names_right.end() == std::find(join_key_names_right.begin(), join_key_names_right.end(), key->getAliasOrColumnName()))
|
||||
join_key_names_right.push_back(key->getAliasOrColumnName());
|
||||
else
|
||||
throw Exception("Duplicate column " + key->getAliasOrColumnName() + " in USING list", ErrorCodes::DUPLICATE_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto i : ext::range(0, nested_result_sample.columns()))
|
||||
{
|
||||
const auto & col = nested_result_sample.getByPosition(i);
|
||||
if (!join_key_names_right_set.count(col.name))
|
||||
if (join_key_names_right.end() == std::find(join_key_names_right.begin(), join_key_names_right.end(), col.name)
|
||||
&& !joined_columns.count(col.name)) /// Дублирующиеся столбцы в подзапросе для JOIN-а не имеют смысла.
|
||||
{
|
||||
joined_columns.insert(col.name);
|
||||
joined_columns_name_type.emplace_back(col.name, col.type);
|
||||
}
|
||||
}
|
||||
|
||||
/* for (const auto & name : join_key_names_left_set)
|
||||
/* for (const auto & name : join_key_names_left)
|
||||
std::cerr << "JOIN key (left): " << name << std::endl;
|
||||
for (const auto & name : join_key_names_right_set)
|
||||
for (const auto & name : join_key_names_right)
|
||||
std::cerr << "JOIN key (right): " << name << std::endl;
|
||||
std::cerr << std::endl;
|
||||
for (const auto & name : joined_columns)
|
||||
|
@ -6,8 +6,11 @@
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
|
||||
#include <DB/DataStreams/MaterializingBlockInputStream.h>
|
||||
#include <DB/DataStreams/NullAndDoCopyBlockInputStream.h>
|
||||
#include <DB/DataStreams/ProhibitColumnsBlockOutputStream.h>
|
||||
#include <DB/DataStreams/MaterializingBlockOutputStream.h>
|
||||
#include <DB/DataStreams/AddingDefaultBlockOutputStream.h>
|
||||
#include <DB/DataStreams/PushingToViewsBlockOutputStream.h>
|
||||
|
||||
#include <DB/Parsers/ASTCreateQuery.h>
|
||||
#include <DB/Parsers/ASTNameTypePair.h>
|
||||
@ -251,11 +254,24 @@ BlockIO InterpreterCreateQuery::executeImpl(bool assume_metadata_exists)
|
||||
/// Если запрос CREATE SELECT, то вставим в таблицу данные
|
||||
if (create.select && storage_name != "View" && (storage_name != "MaterializedView" || create.is_populate))
|
||||
{
|
||||
auto table_lock = res->lockStructure(true);
|
||||
|
||||
/// Также см. InterpreterInsertQuery.
|
||||
BlockOutputStreamPtr out{
|
||||
new ProhibitColumnsBlockOutputStream{
|
||||
new AddingDefaultBlockOutputStream{
|
||||
new MaterializingBlockOutputStream{
|
||||
new PushingToViewsBlockOutputStream{create.database, create.table, context, query_ptr}
|
||||
},
|
||||
columns, column_defaults, context, context.getSettingsRef().strict_insert_defaults
|
||||
},
|
||||
materialized_columns
|
||||
}
|
||||
};
|
||||
|
||||
BlockIO io;
|
||||
io.in_sample = select_sample;
|
||||
io.in = new NullAndDoCopyBlockInputStream(
|
||||
new MaterializingBlockInputStream(interpreter_select->execute().in),
|
||||
res->write(query_ptr));
|
||||
io.in = new NullAndDoCopyBlockInputStream(interpreter_select->execute().in, out);
|
||||
|
||||
return io;
|
||||
}
|
||||
|
@ -98,7 +98,7 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi
|
||||
|
||||
void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_)
|
||||
{
|
||||
if (query.table && typeid_cast<ASTSelectQuery *>(&*query.table))
|
||||
if (query.table && typeid_cast<ASTSelectQuery *>(query.table.get()))
|
||||
{
|
||||
if (table_column_names.empty())
|
||||
{
|
||||
@ -107,10 +107,10 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_)
|
||||
}
|
||||
else
|
||||
{
|
||||
if (query.table && typeid_cast<const ASTFunction *>(&*query.table))
|
||||
if (query.table && typeid_cast<const ASTFunction *>(query.table.get()))
|
||||
{
|
||||
/// Получить табличную функцию
|
||||
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast<const ASTFunction *>(&*query.table)->name, context);
|
||||
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast<const ASTFunction *>(query.table.get())->name, context);
|
||||
/// Выполнить ее и запомнить результат
|
||||
storage = table_function_ptr->execute(query.table, context);
|
||||
}
|
||||
@ -329,7 +329,7 @@ BlockIO InterpreterSelectQuery::execute()
|
||||
executeUnion();
|
||||
|
||||
/// Ограничения на результат, квота на результат, а также колбек для прогресса.
|
||||
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(&*streams[0]))
|
||||
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(streams[0].get()))
|
||||
{
|
||||
/// Ограничения действуют только на конечный результат.
|
||||
if (to_stage == QueryProcessingStage::Complete)
|
||||
@ -590,7 +590,7 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
{
|
||||
transformStreams([&](auto & stream)
|
||||
{
|
||||
if (IProfilingBlockInputStream * p_stream = dynamic_cast<IProfilingBlockInputStream *>(&*stream))
|
||||
if (IProfilingBlockInputStream * p_stream = dynamic_cast<IProfilingBlockInputStream *>(stream.get()))
|
||||
p_stream->enableExtremes();
|
||||
});
|
||||
}
|
||||
@ -651,7 +651,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
/// Список столбцов, которых нужно прочитать, чтобы выполнить запрос.
|
||||
Names required_columns = query_analyzer->getRequiredColumns();
|
||||
|
||||
if (query.table && typeid_cast<ASTSelectQuery *>(&*query.table))
|
||||
if (query.table && typeid_cast<ASTSelectQuery *>(query.table.get()))
|
||||
{
|
||||
/** Для подзапроса не действуют ограничения на максимальный размер результата.
|
||||
* Так как результат поздапроса - ещё не результат всего запроса.
|
||||
@ -792,7 +792,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
|
||||
transformStreams([&](auto & stream)
|
||||
{
|
||||
if (IProfilingBlockInputStream * p_stream = dynamic_cast<IProfilingBlockInputStream *>(&*stream))
|
||||
if (IProfilingBlockInputStream * p_stream = dynamic_cast<IProfilingBlockInputStream *>(stream.get()))
|
||||
{
|
||||
p_stream->setLimits(limits);
|
||||
p_stream->setQuota(quota);
|
||||
@ -1091,9 +1091,38 @@ void InterpreterSelectQuery::executeLimit()
|
||||
/// Если есть LIMIT
|
||||
if (query.limit_length)
|
||||
{
|
||||
/** Редкий случай:
|
||||
* если нет WITH TOTALS и есть подзапрос в FROM, и там на одном из уровней есть WITH TOTALS,
|
||||
* то при использовании LIMIT-а следует читать данные до конца, а не отменять выполнение запроса раньше,
|
||||
* потому что при отмене выполнения запроса, мы не получим данные для totals с удалённого сервера.
|
||||
*/
|
||||
bool always_read_till_end = false;
|
||||
if (!query.group_by_with_totals && query.table && typeid_cast<const ASTSelectQuery *>(query.table.get()))
|
||||
{
|
||||
const ASTSelectQuery * subquery = static_cast<const ASTSelectQuery *>(query.table.get());
|
||||
|
||||
while (subquery->table)
|
||||
{
|
||||
if (subquery->group_by_with_totals)
|
||||
{
|
||||
/** NOTE Можно ещё проверять, что таблица в подзапросе - распределённая, и что она смотрит только на один шард.
|
||||
* В остальных случаях totals будет вычислен на сервере-инициаторе запроса, и читать данные до конца не обязательно.
|
||||
*/
|
||||
|
||||
always_read_till_end = true;
|
||||
break;
|
||||
}
|
||||
|
||||
if (typeid_cast<const ASTSelectQuery *>(subquery->table.get()))
|
||||
subquery = static_cast<const ASTSelectQuery *>(subquery->table.get());
|
||||
else
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
transformStreams([&](auto & stream)
|
||||
{
|
||||
stream = new LimitBlockInputStream(stream, limit_length, limit_offset);
|
||||
stream = new LimitBlockInputStream(stream, limit_length, limit_offset, always_read_till_end);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -376,12 +376,18 @@ void Join::setSampleBlock(const Block & block)
|
||||
|
||||
sample_block_with_columns_to_add = block;
|
||||
|
||||
/// Удаляем из sample_block_with_columns_to_add ключевые столбцы.
|
||||
for (const auto & name : key_names_right)
|
||||
/// Переносим из sample_block_with_columns_to_add ключевые столбцы в sample_block_with_keys, сохраняя порядок.
|
||||
size_t pos = 0;
|
||||
while (pos < sample_block_with_columns_to_add.columns())
|
||||
{
|
||||
size_t pos = sample_block_with_columns_to_add.getPositionByName(name);
|
||||
sample_block_with_keys.insert(sample_block_with_columns_to_add.unsafeGetByPosition(pos));
|
||||
sample_block_with_columns_to_add.erase(pos);
|
||||
const auto & name = sample_block_with_columns_to_add.unsafeGetByPosition(pos).name;
|
||||
if (key_names_right.end() != std::find(key_names_right.begin(), key_names_right.end(), name))
|
||||
{
|
||||
sample_block_with_keys.insert(sample_block_with_columns_to_add.unsafeGetByPosition(pos));
|
||||
sample_block_with_columns_to_add.erase(pos);
|
||||
}
|
||||
else
|
||||
++pos;
|
||||
}
|
||||
|
||||
for (size_t i = 0, size = sample_block_with_columns_to_add.columns(); i < size; ++i)
|
||||
@ -426,7 +432,9 @@ bool Join::insertFromBlock(const Block & block)
|
||||
|
||||
if (getFullness(kind))
|
||||
{
|
||||
/// Переносим ключевые столбцы в начало блока.
|
||||
/** Переносим ключевые столбцы в начало блока.
|
||||
* Именно там их будет ожидать NonJoinedBlockInputStream.
|
||||
*/
|
||||
size_t key_num = 0;
|
||||
for (const auto & name : key_names_right)
|
||||
{
|
||||
@ -810,6 +818,8 @@ void Join::checkTypesOfKeys(const Block & block_left, const Block & block_right)
|
||||
|
||||
void Join::joinBlock(Block & block) const
|
||||
{
|
||||
// std::cerr << "joinBlock: " << block.dumpStructure() << "\n";
|
||||
|
||||
Poco::ScopedReadRWLock lock(rwlock);
|
||||
|
||||
checkTypesOfKeys(block, sample_block_with_keys);
|
||||
@ -917,6 +927,8 @@ public:
|
||||
|
||||
result_sample_block = left_sample_block;
|
||||
|
||||
// std::cerr << result_sample_block.dumpStructure() << "\n";
|
||||
|
||||
/// Добавляем в блок новые столбцы.
|
||||
for (size_t i = 0; i < num_columns_right; ++i)
|
||||
{
|
||||
@ -932,10 +944,11 @@ public:
|
||||
{
|
||||
const String & name = left_sample_block.getByPosition(i).name;
|
||||
|
||||
if (parent.key_names_left.end() == std::find(parent.key_names_left.begin(), parent.key_names_left.end(), name))
|
||||
auto found_key_column = std::find(parent.key_names_left.begin(), parent.key_names_left.end(), name);
|
||||
if (parent.key_names_left.end() == found_key_column)
|
||||
column_numbers_left.push_back(i);
|
||||
else
|
||||
column_numbers_keys_and_right.push_back(i);
|
||||
column_numbers_keys_and_right.push_back(found_key_column - parent.key_names_left.begin());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < num_columns_right; ++i)
|
||||
@ -1046,8 +1059,6 @@ private:
|
||||
|
||||
for (; it != end; ++it)
|
||||
{
|
||||
// std::cerr << it->second.getUsed() << "\n";
|
||||
|
||||
if (it->second.getUsed())
|
||||
continue;
|
||||
|
||||
|
@ -56,6 +56,9 @@ void LogicalExpressionsOptimizer::optimizeDisjunctiveEqualityChains()
|
||||
|
||||
void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
||||
{
|
||||
if (select_query->attributes & IAST::IsVisited)
|
||||
return;
|
||||
|
||||
using Edge = std::pair<IAST *, IAST *>;
|
||||
std::deque<Edge> to_visit;
|
||||
|
||||
@ -67,7 +70,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
||||
auto to_node = edge.second;
|
||||
|
||||
to_visit.pop_back();
|
||||
to_node->is_visited = true;
|
||||
to_node->attributes |= IAST::IsVisited;
|
||||
|
||||
bool found_chain = false;
|
||||
|
||||
@ -116,7 +119,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
||||
{
|
||||
if (typeid_cast<ASTSelectQuery *>(&*child) == nullptr)
|
||||
{
|
||||
if (!child->is_visited)
|
||||
if (!(child->attributes & IAST::IsVisited))
|
||||
to_visit.push_back(Edge(to_node, &*child));
|
||||
else
|
||||
{
|
||||
@ -133,8 +136,6 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
||||
}
|
||||
}
|
||||
|
||||
select_query->clearVisited();
|
||||
|
||||
for (auto & chain : disjunctive_equality_chains_map)
|
||||
{
|
||||
auto & equalities = chain.second;
|
||||
|
@ -6,26 +6,33 @@ namespace DB
|
||||
|
||||
ProcessList::EntryPtr ProcessList::insert(
|
||||
const String & query_, const String & user_, const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
size_t max_memory_usage, size_t max_wait_milliseconds, bool replace_running_query, QueryPriorities::Priority priority)
|
||||
const Settings & settings)
|
||||
{
|
||||
EntryPtr res;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
if (max_size && cur_size >= max_size && (!max_wait_milliseconds || !have_space.tryWait(mutex, max_wait_milliseconds)))
|
||||
if (max_size && cur_size >= max_size
|
||||
&& (!settings.queue_max_wait_ms.totalMilliseconds() || !have_space.tryWait(mutex, settings.queue_max_wait_ms.totalMilliseconds())))
|
||||
throw Exception("Too much simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES);
|
||||
|
||||
if (!query_id_.empty())
|
||||
{
|
||||
UserToQueries::iterator queries = user_to_queries.find(user_);
|
||||
UserToQueries::iterator queries = user_to_queries.find(user_);
|
||||
|
||||
if (queries != user_to_queries.end())
|
||||
if (queries != user_to_queries.end())
|
||||
{
|
||||
if (settings.max_concurrent_queries_for_user && queries->second.size() >= settings.max_concurrent_queries_for_user)
|
||||
throw Exception("Too much simultaneous queries for user " + user_
|
||||
+ ". Current: " + toString(queries->second.size())
|
||||
+ ", maximum: " + toString(settings.max_concurrent_queries_for_user),
|
||||
ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES);
|
||||
|
||||
if (!query_id_.empty())
|
||||
{
|
||||
QueryToElement::iterator element = queries->second.find(query_id_);
|
||||
if (element != queries->second.end())
|
||||
{
|
||||
if (!replace_running_query)
|
||||
if (!settings.replace_running_query)
|
||||
throw Exception("Query with id = " + query_id_ + " is already running.",
|
||||
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
|
||||
element->second->is_cancelled = true;
|
||||
@ -38,7 +45,7 @@ ProcessList::EntryPtr ProcessList::insert(
|
||||
++cur_size;
|
||||
|
||||
res.reset(new Entry(*this, cont.emplace(cont.end(),
|
||||
query_, user_, query_id_, ip_address_, max_memory_usage, priorities.insert(priority))));
|
||||
query_, user_, query_id_, ip_address_, settings.limits.max_memory_usage, priorities.insert(settings.priority))));
|
||||
|
||||
if (!query_id_.empty())
|
||||
user_to_queries[user_][query_id_] = &res->get();
|
||||
@ -69,4 +76,34 @@ ProcessListEntry::~ProcessListEntry()
|
||||
parent.have_space.signal();
|
||||
}
|
||||
|
||||
|
||||
void ProcessList::addTemporaryTable(ProcessListElement & elem, const String & table_name, StoragePtr storage)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
elem.temporary_tables[table_name] = storage;
|
||||
}
|
||||
|
||||
|
||||
StoragePtr ProcessList::tryGetTemporaryTable(const String & query_id, const String & table_name) const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
/// NOTE Ищем по всем user-ам. То есть, нет изоляции, и сложность O(users).
|
||||
for (const auto & user_queries : user_to_queries)
|
||||
{
|
||||
auto it = user_queries.second.find(query_id);
|
||||
if (user_queries.second.end() == it)
|
||||
continue;
|
||||
|
||||
auto jt = (*it->second).temporary_tables.find(table_name);
|
||||
if ((*it->second).temporary_tables.end() == jt)
|
||||
continue;
|
||||
|
||||
return jt->second;
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -261,7 +261,7 @@ void QueryLog::flush()
|
||||
block.unsafeGetByPosition(i++).column.get()->insertData(elem.query_id.data(), elem.query_id.size());
|
||||
}
|
||||
|
||||
BlockOutputStreamPtr stream = table->write(nullptr);
|
||||
BlockOutputStreamPtr stream = table->write({}, {});
|
||||
|
||||
stream->writePrefix();
|
||||
stream->write(block);
|
||||
|
@ -83,6 +83,20 @@ void Settings::setProfile(const String & profile_name, Poco::Util::AbstractConfi
|
||||
}
|
||||
}
|
||||
|
||||
void Settings::loadSettingsFromConfig(const String & path, Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
if (!config.has(path))
|
||||
throw Exception("There is no path '" + path + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(path, config_keys);
|
||||
|
||||
for (const std::string & key : config_keys)
|
||||
{
|
||||
set(key, config.getString(path + "." + key));
|
||||
}
|
||||
}
|
||||
|
||||
/// Прочитать настройки из буфера. Они записаны как набор name-value пар, идущих подряд, заканчивающихся пустым name.
|
||||
/// Если выставлен флаг check_readonly, в настройках выставлено readonly, но пришли какие-то изменения кинуть исключение.
|
||||
void Settings::deserialize(ReadBuffer & buf)
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user