This commit is contained in:
Roman Peshkurov 2015-09-08 19:45:48 +03:00
commit 0cac5e3744
83 changed files with 4042 additions and 632 deletions

View File

@ -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;
}

View File

@ -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));
}
};
}

View 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();
};
}

View File

@ -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()

View File

@ -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

View File

@ -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

View File

@ -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)

View File

@ -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;

View File

@ -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

View File

@ -286,6 +286,7 @@ namespace ErrorCodes
NO_AVAILABLE_DATA = 280,
DICTIONARY_IS_EMPTY = 281,
INCORRECT_INDEX = 282,
UNKNOWN_GLOBAL_SUBQUERIES_METHOD = 284,
KEEPER_EXCEPTION = 999,
POCO_EXCEPTION = 1000,

View File

@ -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;
};
}

View File

@ -12,6 +12,19 @@ namespace DB
* Это экономит оперативку в случае использования двухуровневой агрегации, где в каждом потоке будет до 256 блоков с частями результата.
*
* Агрегатные функции в блоках не должны быть финализированы, чтобы их состояния можно было объединить.
*
* Замечания:
*
* На хорошей сети (10Gbit) может работать заметно медленнее, так как чтения блоков с разных
* удалённых серверов делаются последовательно, при этом, чтение упирается в CPU.
* Это несложно исправить.
*
* Также, чтения и вычисления (слияние состояний) делаются по очереди.
* Есть возможность делать чтения асинхронно - при этом будет расходоваться в два раза больше памяти, но всё-равно немного.
* Это можно сделать с помощью UnionBlockInputStream.
*
* Можно держать в памяти не по одному блоку из каждого источника, а по несколько, и распараллелить мердж.
* При этом будет расходоваться кратно больше оперативки.
*/
class MergingAggregatedMemoryEfficientBlockInputStream : public IProfilingBlockInputStream
{
@ -19,20 +32,18 @@ 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_)
final(final_),
inputs(inputs_.begin(), inputs_.end())
{
children = inputs_;
current_blocks.resize(children.size());
overflow_blocks.resize(children.size());
is_exhausted.resize(children.size());
}
String getName() const override { return "MergingAggregatedMemorySavvy"; }
String getName() const override { return "MergingAggregatedMemoryEfficient"; }
String getID() const override
{
std::stringstream res;
res << "MergingAggregatedMemorySavvy(" << aggregator.getID();
res << "MergingAggregatedMemoryEfficient(" << aggregator.getID();
for (size_t i = 0, size = children.size(); i < size; ++i)
res << ", " << children.back()->getID();
res << ")";
@ -43,68 +54,222 @@ protected:
Block readImpl() override
{
/// Если child - RemoteBlockInputStream, то отправляет запрос на все удалённые серверы, инициируя вычисления.
if (current_bucket_num == -1)
if (!started)
{
started = true;
for (auto & child : children)
child->readPrefix();
}
/// Всё прочитали.
if (current_bucket_num > 255)
return {};
/** Имеем несколько источников.
* Из каждого из них могут приходить следующие данные:
*
* 1. Блок, с указанным bucket_num.
* Это значит, что на удалённом сервере, данные были разрезаны по корзинам.
* И данные для одного bucket_num с разных серверов можно независимо объединять.
* При этом, даннные для разных bucket_num будут идти по возрастанию.
*
* 2. Блок без указания bucket_num.
* Это значит, что на удалённом сервере, данные не были разрезаны по корзинам.
* В случае, когда со всех серверов прийдут такие данные, их можно всех объединить.
* А если с другой части серверов прийдут данные, разрезанные по корзинам,
* то данные, не разрезанные по корзинам, нужно сначала разрезать, а потом объединять.
*
* 3. Блоки с указанием is_overflows.
* Это дополнительные данные для строк, не прошедших через max_rows_to_group_by.
* Они должны объединяться друг с другом отдельно.
*/
/// Читаем следующие блоки для current_bucket_num
for (size_t i = 0, size = children.size(); i < size; ++i)
constexpr size_t NUM_BUCKETS = 256;
++current_bucket_num;
for (auto & input : inputs)
{
while (!is_exhausted[i] && (!current_blocks[i] || current_blocks[i].info.bucket_num < current_bucket_num))
{
current_blocks[i] = children[i]->read();
if (input.is_exhausted)
continue;
if (!current_blocks[i])
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)
{
is_exhausted[i] = true;
// std::cerr << "input is exhausted\n";
input.is_exhausted = true;
break;
}
else if (current_blocks[i].info.is_overflows)
if (block.info.bucket_num != -1)
{
overflow_blocks[i].swap(current_blocks[i]);
/// Один из разрезанных блоков для двухуровневых данных.
// 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;
}
}
/// Может быть, нет блоков для 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;
while (true)
{
if (current_bucket_num == NUM_BUCKETS)
{
/// Обработали все основные данные. Остались, возможно, только overflows-блоки.
// std::cerr << "at end\n";
current_bucket_num = min_bucket_num;
if (has_overflows)
{
// std::cerr << "merging overflows\n";
/// Все потоки исчерпаны.
if (current_bucket_num > 255)
return {}; /// TODO overflow_blocks.
has_overflows = false;
BlocksList blocks_to_merge;
/// TODO Если есть single_level и two_level блоки.
for (auto & input : inputs)
if (input.overflow_block)
blocks_to_merge.emplace_back(std::move(input.overflow_block));
/// Объединяем все блоки с current_bucket_num.
return aggregator.mergeBlocks(blocks_to_merge, final);
}
else
return {};
}
else if (has_two_level)
{
/** Есть двухуровневые данные.
* Будем обрабатывать номера корзин по возрастанию.
* Найдём минимальный номер корзины, для которой есть данные,
* затем померджим эти данные.
*/
// std::cerr << "has two level\n";
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]));
int min_bucket_num = NUM_BUCKETS;
Block res = aggregator.mergeBlocks(blocks_to_merge, final);
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;
++current_bucket_num;
return res;
/// Ещё не разрезанный по корзинам блок. Разрезаем его и кладём результат в 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);
/** Нельзя уничтожать исходный блок.
* Потому что он владеет Arena с состояниями агрегатных функций,
* а splitted_blocks ей не владеют, но ссылаются на эти состояния.
*/
}
/// Блоки, которые мы получили разрезанием одноуровневых блоков.
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);
}
}
}
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;
};
}

View File

@ -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)

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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;

View 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);
}
}

View File

@ -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;
}
}
};

View File

@ -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>

View File

@ -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;
}

View File

@ -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)};
});
}
}

View File

@ -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 кинет исключение.
}

View File

@ -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)
{
}

View File

@ -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;
}

View File

@ -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;

View File

@ -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;
};

View File

@ -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;

View File

@ -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_,
@ -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;
};
}

View File

@ -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,9 @@ struct Settings
\
/** Логгировать запросы и писать лог в системную таблицу. */ \
M(SettingBool, log_queries, 0) \
\
/** Схема выполнения GLOBAL-подзапросов. */ \
M(SettingGlobalSubqueriesMethod, global_subqueries_method, GlobalSubqueriesMethod::PUSH) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

@ -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,72 @@ struct SettingCompressionMethod
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);
}
};
}

View File

@ -308,7 +308,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>;
}

View 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>;
}

View File

@ -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,

View File

@ -0,0 +1,435 @@
#pragma once
#include <DB/Core/NamesAndTypes.h>
#include <DB/Storages/MergeTree/RangesInDataPart.h>
#include <statdaemons/ext/range.hpp>
#include <mutex>
namespace DB
{
struct MergeTreeReadTask
{
MergeTreeData::DataPartPtr data_part;
MarkRanges mark_ranges;
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;
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>;
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;
/// Восстановим порядок отрезков.
std::reverse(thread_task.ranges.begin(), thread_task.ranges.end());
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;
}
}
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)
{
/// Оставим отрезки перечисленными справа налево для удобства.
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().
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);
}
}
}
}
if (all_column_files_missing)
{
addMinimumSizeColumn(part, columns);
/// correctly report added column
injected_columns.insert(columns.back());
}
return injected_columns;
}
/** Добавить столбец минимального размера.
* Используется в случае, когда ни один столбец не нужен, но нужно хотя бы знать количество строк.
* Добавляет в columns.
*/
void addMinimumSizeColumn(const MergeTreeData::DataPartPtr & part, Names & columns) const
{
const auto get_column_size = [this, &part] (const String & name) {
const auto & files = 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 = data.getColumnsList();
const NameAndTypePair * minimum_size_column = nullptr;
auto minimum_size = std::numeric_limits<size_t>::max();
for (const auto & column : storage_columns)
{
if (!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
};
columns.push_back(minimum_size_column->name);
}
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<bool> per_part_remove_prewhere_column;
std::vector<bool> per_part_should_reorder;
struct part_t
{
MergeTreeData::DataPartPtr data_part;
std::size_t part_index_in_query;
};
std::vector<part_t> parts;
struct thread_task_t
{
struct part_index_and_range_t
{
std::size_t part_idx;
MarkRanges ranges;
};
std::vector<part_index_and_range_t> parts_and_ranges;
std::vector<std::size_t> sum_marks_in_parts;
};
std::vector<thread_task_t> threads_tasks;
std::unordered_set<std::size_t> remaining_thread_tasks;
mutable std::mutex mutex;
};
using MergeTreeReadPoolPtr = std::shared_ptr<MergeTreeReadPool>;
}

View File

@ -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,19 +18,6 @@
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-и быстро, не выбрасывая содержимое буфера.
@ -44,11 +32,23 @@ public:
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_)
: uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_), storage(storage_),
aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_)
{
reconf(path_, data_part, columns_, all_mark_ranges);
}
void reconf(
const String & path, const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns,
const MarkRanges & all_mark_ranges)
{
this->path = path;
this->data_part = data_part;
this->part_name = data_part->name;
this->columns = columns;
this->all_mark_ranges = all_mark_ranges;
this->streams.clear();
try
{
if (!Poco::File(path).exists())
@ -74,20 +74,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 +114,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);
}
/// Более хорошая диагностика.
throw Exception(e.message() + "\n(while reading from part " + path + " from mark " + toString(from_mark) + " to "
@ -151,60 +133,13 @@ public:
}
}
/** Добавить столбец минимального размера.
* Используется в случае, когда ни один столбец не нужен, но нужно хотя бы знать количество строк.
* Добавляет в 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 +155,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 +214,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 +283,21 @@ private:
typedef std::map<std::string, std::unique_ptr<Stream> > FileStreams;
/// Используется в качестве подсказки, чтобы уменьшить количество реаллокаций при создании столбца переменной длины.
std::map<std::string, double> avg_value_size_hints;
String path;
const MergeTreeData::DataPartPtr & data_part;
MergeTreeData::DataPartPtr data_part;
String part_name;
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 +320,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 +389,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 +400,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 +485,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 +494,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);
}
}

View File

@ -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;

View File

@ -0,0 +1,342 @@
#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
{
class MergeTreeThreadBlockInputStream : public IProfilingBlockInputStream
{
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:
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
{
reader->reconf(path, task->data_part, task->columns, task->mark_ranges);
if (prewhere_actions)
pre_reader->reconf(path, task->data_part, task->pre_columns, task->mark_ranges);
}
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;
};
}

View 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>;
}

View File

@ -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
View 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()

View 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()

View File

@ -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",

View 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;
}
}

View File

@ -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
{

View File

@ -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());

View File

@ -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);

View File

@ -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);

View File

@ -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>();

View File

@ -8,6 +8,7 @@ void registerFunctionsRandom(FunctionFactory & factory)
{
factory.registerFunction<FunctionRand>();
factory.registerFunction<FunctionRand64>();
factory.registerFunction<FunctionRandConstant>();
}
}

View File

@ -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);

View File

@ -58,7 +58,7 @@ void AggregatedDataVariants::convertToTwoLevel()
}
void Aggregator::initialize(Block & block)
void Aggregator::initialize(const Block & block)
{
if (isCancelled())
return;
@ -1735,17 +1735,162 @@ 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.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)
{
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();
/// Для всех строчек.
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);
Block & dst = destinations[bucket];
if (unlikely(!dst))
{
dst = source.cloneEmpty();
dst.info.bucket_num = bucket;
}
for (size_t j = 0; j < columns; ++j)
dst.unsafeGetByPosition(j).column.get()->insertFrom(*source.unsafeGetByPosition(j).column.get(), i);
}
}
std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
{
if (!block)
return {};
initialize(block);
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);
#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;
}

View File

@ -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();

View File

@ -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'",

View File

@ -1,3 +1,5 @@
#include <Poco/Util/Application.h>
#include <DB/DataTypes/FieldToDataType.h>
#include <DB/Parsers/ASTFunction.h>
@ -886,8 +888,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 +936,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 +964,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();
@ -1743,15 +1844,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 +2236,31 @@ 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.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)

View File

@ -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);
});
}
}

View File

@ -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;

View File

@ -69,4 +69,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 {};
}
}

View File

@ -259,7 +259,7 @@ void ITableDeclaration::check(const Block & block, bool need_all) const
const ColumnWithTypeAndName & column = block.getByPosition(i);
if (names_in_block.count(column.name))
throw Exception("Duplicate column " + column.name + " in block",
throw Exception("Duplicate column " + column.name + " in block",
ErrorCodes::DUPLICATE_COLUMN);
names_in_block.insert(column.name);

View File

@ -1,6 +1,7 @@
#include <DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <DB/Storages/MergeTree/MergeTreeBlockInputStream.h>
#include <DB/Interpreters/ExpressionAnalyzer.h>
#include <DB/Storages/MergeTree/MergeTreeReadPool.h>
#include <DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h>
#include <DB/DataStreams/FilterBlockInputStream.h>
@ -346,9 +347,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
const Names & virt_columns,
const Settings & settings)
{
const size_t min_marks_for_concurrent_read =
const std::size_t min_marks_for_concurrent_read =
(settings.merge_tree_min_rows_for_concurrent_read + data.index_granularity - 1) / data.index_granularity;
const size_t max_marks_to_use_cache =
const std::size_t max_marks_to_use_cache =
(settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity;
/// Посчитаем засечки для каждого куска.
@ -370,7 +371,27 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
BlockInputStreams res;
if (sum_marks > 0)
if (sum_marks > 0 && settings.merge_tree_uniform_read_distribution == 1)
{
MergeTreeReadPoolPtr pool = std::make_shared<MergeTreeReadPool>(
threads, sum_marks, min_marks_for_concurrent_read, parts, data, prewhere_actions, prewhere_column, true,
column_names);
for (std::size_t i = 0; i < threads; ++i)
res.emplace_back(new MergeTreeThreadBlockInputStream{
i, pool, min_marks_for_concurrent_read, max_block_size, data, use_uncompressed_cache, prewhere_actions,
prewhere_column, settings, virt_columns
});
/// Оценим общее количество строк - для прогресс-бара.
const std::size_t total_rows = data.index_granularity * sum_marks;
/// Выставим приблизительное количество строк только для первого источника
static_cast<IProfilingBlockInputStream &>(*res.front()).setTotalRowsApprox(total_rows);
LOG_TRACE(log, "Reading approx. " << total_rows);
}
else if (sum_marks > 0)
{
const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1;
@ -468,7 +489,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal
const Settings & settings,
const Context & context)
{
size_t max_marks_to_use_cache = (settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity;
const size_t max_marks_to_use_cache =
(settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity;
const size_t min_marks_for_read_task =
(settings.merge_tree_min_rows_for_concurrent_read + data.index_granularity - 1) / data.index_granularity;
size_t sum_marks = 0;
for (size_t i = 0; i < parts.size(); ++i)
@ -480,26 +504,55 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal
BlockInputStreams to_merge;
for (size_t part_index = 0; part_index < parts.size(); ++part_index)
if (settings.merge_tree_uniform_read_distribution == 1)
{
RangesInDataPart & part = parts[part_index];
MergeTreeReadPoolPtr pool = std::make_shared<MergeTreeReadPool>(
parts.size(), sum_marks, min_marks_for_read_task, parts, data, prewhere_actions, prewhere_column, true,
column_names, true);
BlockInputStreamPtr source_stream = new MergeTreeBlockInputStream(
data.getFullPath() + part.data_part->name + '/', max_block_size, column_names, data,
part.data_part, part.ranges, use_uncompressed_cache,
prewhere_actions, prewhere_column, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size);
for (const String & virt_column : virt_columns)
for (const auto i : ext::range(0, parts.size()))
{
if (virt_column == "_part")
source_stream = new AddingConstColumnBlockInputStream<String>(
source_stream, new DataTypeString, part.data_part->name, "_part");
else if (virt_column == "_part_index")
source_stream = new AddingConstColumnBlockInputStream<UInt64>(
source_stream, new DataTypeUInt64, part.part_index_in_query, "_part_index");
BlockInputStreamPtr source_stream{
new MergeTreeThreadBlockInputStream{
i, pool, min_marks_for_read_task, max_block_size, data, use_uncompressed_cache, prewhere_actions,
prewhere_column, settings, virt_columns
}
};
to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression()));
}
to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression()));
/// Оценим общее количество строк - для прогресс-бара.
const std::size_t total_rows = data.index_granularity * sum_marks;
/// Выставим приблизительное количество строк только для первого источника
static_cast<IProfilingBlockInputStream &>(*to_merge.front()).setTotalRowsApprox(total_rows);
LOG_TRACE(log, "Reading approx. " << total_rows);
}
else
{
for (size_t part_index = 0; part_index < parts.size(); ++part_index)
{
RangesInDataPart & part = parts[part_index];
BlockInputStreamPtr source_stream = new MergeTreeBlockInputStream(
data.getFullPath() + part.data_part->name + '/', max_block_size, column_names, data,
part.data_part, part.ranges, use_uncompressed_cache,
prewhere_actions, prewhere_column, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size);
for (const String & virt_column : virt_columns)
{
if (virt_column == "_part")
source_stream = new AddingConstColumnBlockInputStream<String>(
source_stream, new DataTypeString, part.data_part->name, "_part");
else if (virt_column == "_part_index")
source_stream = new AddingConstColumnBlockInputStream<UInt64>(
source_stream, new DataTypeUInt64, part.part_index_in_query, "_part_index");
}
to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression()));
}
}
BlockInputStreams res;

View File

@ -149,8 +149,11 @@ static void appendBlock(const Block & from, Block & to)
throw Exception("Cannot append block to another: different type of columns at index " + toString(column_no)
+ ". Block 1: " + from.dumpStructure() + ". Block 2: " + to.dumpStructure(), ErrorCodes::BLOCKS_HAS_DIFFERENT_STRUCTURE);
for (size_t row_no = 0; row_no < rows; ++row_no)
col_to.insertFrom(col_from, row_no);
if (col_to.empty())
to.getByPosition(column_no).column = col_from.clone();
else
for (size_t row_no = 0; row_no < rows; ++row_no)
col_to.insertFrom(col_from, row_no);
}
}
@ -243,24 +246,18 @@ private:
buffer.data = sorted_block.cloneEmpty();
}
/// Если после вставки в буфер, ограничения будут превышены, то будем сбрасывать буфер.
/** Если после вставки в буфер, ограничения будут превышены, то будем сбрасывать буфер.
* Это также защищает от неограниченного потребления оперативки, так как в случае невозможности записать в таблицу,
* будет выкинуто исключение, а новые данные не будут добавлены в буфер.
*/
if (storage.checkThresholds(buffer, time(0), sorted_block.rowsInFirstColumn(), sorted_block.bytes()))
{
/// Вытащим из буфера блок, заменим буфер на пустой. После этого можно разблокировать mutex.
Block block_to_write;
buffer.data.swap(block_to_write);
buffer.first_write_time = 0;
lock.unlock();
if (!storage.no_destination)
{
auto destination = storage.context.tryGetTable(storage.destination_database, storage.destination_table);
appendBlock(sorted_block, block_to_write);
storage.writeBlockToDestination(block_to_write, destination);
}
storage.flushBuffer(buffer, false);
lock.lock();
}
else
appendBlock(sorted_block, buffer.data);
appendBlock(sorted_block, buffer.data);
}
};
@ -278,7 +275,14 @@ void StorageBuffer::shutdown()
if (flush_thread.joinable())
flush_thread.join();
optimize(context.getSettings());
try
{
optimize(context.getSettings());
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
@ -331,8 +335,16 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds)
{
std::lock_guard<std::mutex> lock(buffer.mutex);
if (check_thresholds && !checkThresholds(buffer, current_time))
return;
if (check_thresholds)
{
if (!checkThresholds(buffer, current_time))
return;
}
else
{
if (buffer.data.rowsInFirstColumn() == 0)
return;
}
buffer.data.swap(block_to_write);
buffer.first_write_time = 0;
@ -357,10 +369,11 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds)
* Замечание: остаётся проблема - из-за того, что в разных попытках вставляются разные блоки,
* теряется идемпотентность вставки в ReplicatedMergeTree.
*/
appendBlock(block_to_write, buffer.data);
buffer.data.swap(block_to_write);
appendBlock(buffer.data, block_to_write);
}
buffer.data.swap(block_to_write);
if (!buffer.first_write_time)
buffer.first_write_time = current_time;

View File

@ -168,7 +168,10 @@ BlockInputStreams StorageDistributed::read(
if (settings.limits.max_network_bandwidth)
throttler.reset(new Throttler(settings.limits.max_network_bandwidth));
Tables external_tables = context.getExternalTables();
Tables external_tables;
if (settings.global_subqueries_method == GlobalSubqueriesMethod::PUSH)
external_tables = context.getExternalTables();
/// Цикл по шардам.
for (auto & conn_pool : cluster.pools)

View File

@ -85,7 +85,7 @@ int main(int argc, char ** argv)
DB::WriteBufferFromOStream out_buf(std::cout);
DB::LimitBlockInputStream in_limit(in, 10);
DB::LimitBlockInputStream in_limit(in, 10, 0);
DB::RowOutputStreamPtr output_ = new DB::TabSeparatedRowOutputStream(out_buf, sample);
DB::BlockOutputStreamFromRowOutputStream output(output_);

View File

@ -16,3 +16,79 @@
1
1
1
00000000000000000000FFFF4D583737
00000000000000000000FFFF4D583737
00000000000000000000FFFF7F000001
00000000000000000000FFFFC1FC110A
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f000
2001:db8:ac10:fe01:feed:babe:cafe:0
2001:db8:ac10:fe01:feed:babe:ca00:0
2001:db8:ac10:fe01:feed:babe::
2001:db8:ac10:fe01:feed:ba00::
2001:db8:ac10:fe01:feed::
2001:db8:ac10:fe01:fe00::
2001:db8:ac10:fe01::
2001:db8:ac10:fe00::
2001:db8:ac10::
2001:db8:ac00::
2001:db8::
2001:d00::
2001::
2000::
::
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
2001:db8:ac10:fe01:feed:babe:cafe:f00d
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.10
::ffff:193.252.17.0
::ffff:193.252.0.0
::ffff:193.0.0.0
::ffff:0.0.0.0
::ff00:0:0
::
::
::
::
::
::
::
::
::
::
::
2001:db8:ac10:fe01:abad:babe:fa00:0
2001:db8:ac10:fe01:dead:c0de:ca00:0
2001:db8:ac10:fe01:feed:babe:ca00:0
::ffff:77.0.0.0
::ffff:127.0.0.0
::ffff:193.0.0.0

View File

@ -17,3 +17,109 @@ select IPv6StringToNum('') == toFixedString(materialize(''), 16);
select IPv6StringToNum(materialize('')) == toFixedString(materialize(''), 16);
select IPv6StringToNum('not an ip string') == toFixedString(materialize(''), 16);
select IPv6StringToNum(materialize('not an ip string')) == toFixedString(materialize(''), 16);
/* IPv4ToIPv6 */
SELECT hex(IPv4ToIPv6(1297626935));
/* Тест с таблицей */
DROP TABLE IF EXISTS test.addresses;
CREATE TABLE test.addresses(addr UInt32) ENGINE = Memory;
INSERT INTO test.addresses(addr) VALUES (1297626935), (2130706433), (3254522122);
SELECT hex(IPv4ToIPv6(addr)) FROM test.addresses ORDER BY addr ASC;
/* cutIPv6 */
/* Реальный IPv6-адрес */
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 1, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 2, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 3, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 4, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 5, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 6, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 7, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 8, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 9, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 10, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 11, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 12, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 13, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 14, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 15, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 16, 0);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 1);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 2);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 3);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 4);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 5);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 6);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 7);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 8);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 9);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 10);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 11);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 12);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 13);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 14);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 15);
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 16);
/* IPv4-mapped IPv6-адрес */
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 1, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 2, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 3, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 4, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 5, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 6, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 7, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 8, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 9, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 10, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 11, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 12, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 13, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 14, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 15, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 16, 0);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 1);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 2);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 3);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 4);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 5);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 6);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 7);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 8);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 9);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 10);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 11);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 12);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 13);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 14);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 15);
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 16);
/* Тест с таблицами */
/* Реальные IPv6-адреса */
DROP TABLE IF EXISTS test.addresses;
CREATE TABLE test.addresses(addr String) ENGINE = Memory;
INSERT INTO test.addresses(addr) VALUES ('20010DB8AC10FE01FEEDBABECAFEF00D'), ('20010DB8AC10FE01DEADC0DECAFED00D'), ('20010DB8AC10FE01ABADBABEFACEB00C');
SELECT cutIPv6(toFixedString(unhex(addr), 16), 3, 0) FROM test.addresses ORDER BY addr ASC;
/* IPv4-mapped IPv6-адреса */
DROP TABLE IF EXISTS test.addresses;
CREATE TABLE test.addresses(addr String) ENGINE = Memory;
INSERT INTO test.addresses(addr) VALUES ('00000000000000000000FFFFC1FC110A'), ('00000000000000000000FFFF4D583737'), ('00000000000000000000FFFF7F000001');
SELECT cutIPv6(toFixedString(unhex(addr), 16), 0, 3) FROM test.addresses ORDER BY addr ASC;

View File

@ -206,6 +206,7 @@
31 53574
35 55022
36 53961
1
1 1
3 1
6 1
@ -239,25 +240,25 @@
31 162
35 162
36 162
0 54226
1 54034
3 54016
6 53982
7 54076
9 54218
10 27075
11 54093
13 54108
14 54096
17 54294
19 54070
20 54028
21 54170
22 54106
26 54103
31 54050
35 54130
36 53868
0 53901
1 54056
3 53999
6 54129
7 54213
9 53853
10 26975
11 54064
13 53963
14 53997
17 54129
19 53923
20 53958
21 54117
22 54150
26 54047
31 53948
35 53931
36 53982
0.125 1
0.5 1
0.05 1
@ -291,25 +292,25 @@
0.043 162
0.037 162
0.071 162
0.045 54170
0.125 54076
0.5 54034
0.05 54070
0.143 53982
0.091 27075
0.056 54294
0.048 54028
0.083 54093
0.25 54016
1 54226
0.1 54218
0.028 54130
0.027 53868
0.031 54050
0.067 54096
0.043 54106
0.037 54103
0.071 54108
0.045 54117
0.125 54213
0.5 54056
0.05 53923
0.143 54129
0.091 26975
0.056 54129
0.048 53958
0.083 54064
0.25 53999
1 53901
0.1 53853
0.028 53931
0.027 53982
0.031 53948
0.067 53997
0.043 54150
0.037 54047
0.071 53963
0.5 1
0.05 1
0.25 1
@ -343,25 +344,25 @@
0.037 162
0.1 163
1 162
0.5 54034
0.05 54070
0.25 54016
0.048 54028
0.091 27075
0.043 54106
0.071 54108
0.083 54093
0.125 54076
0.031 54050
0.143 53982
0.028 54130
0.067 54096
0.045 54170
0.027 53868
0.056 54294
0.037 54103
0.1 54218
1 54226
0.5 54056
0.05 53923
0.25 53999
0.048 53958
0.091 26975
0.043 54150
0.071 53963
0.083 54064
0.125 54213
0.031 53948
0.143 54129
0.028 53931
0.067 53997
0.045 54117
0.027 53982
0.056 54129
0.037 54047
0.1 53853
1 53901
1 1
3 1
6 1
@ -414,3 +415,4 @@
31 54074
35 54153
36 53999
1

View File

@ -16,6 +16,8 @@ SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS
SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
SELECT uniqHLL12(dummy) FROM remote('127.0.0.{1,2}', system.one);
/* uniqCombined */
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
@ -33,3 +35,5 @@ SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
SELECT uniqCombined(dummy) FROM remote('127.0.0.{1,2}', system.one);

View File

@ -0,0 +1,2 @@
SELECT k, a FROM (SELECT 42 AS k FROM remote('127.0.0.1', system.one)) GLOBAL ALL FULL OUTER JOIN (SELECT 42 AS k, 1 AS a, a) USING k;
SELECT 1 FROM remote('127.0.0.1', system.one) WHERE (1, 1) GLOBAL IN (SELECT 1 AS a, a);

View File

@ -0,0 +1,11 @@
1
1
1
1
1
1
1
1
1
0
0

View File

@ -0,0 +1,11 @@
SELECT 'abcdef' LIKE '%abc%def%';
SELECT 'abctdef' LIKE '%abc%def%';
SELECT 'abc\ndef' LIKE '%abc%def%';
SELECT 'abc\ntdef' LIKE '%abc%def%';
SELECT 'abct\ndef' LIKE '%abc%def%';
SELECT 'abc\n\ndef' LIKE '%abc%def%';
SELECT 'abc\n\ntdef' LIKE '%abc%def%';
SELECT 'abc\nt\ndef' LIKE '%abc%def%';
SELECT 'abct\n\ndef' LIKE '%abc%def%';
SELECT 'ab\ndef' LIKE '%abc%def%';
SELECT 'abc\nef' LIKE '%abc%def%';

View File

@ -0,0 +1,6 @@
2 3000
2 3000
1 2000
2 3000
1 2000
2 3000

View File

@ -0,0 +1,6 @@
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING a, b;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING b, a;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) USING a, b;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) USING b, a;

View File

@ -0,0 +1 @@
SELECT x FROM (SELECT count() AS x FROM remote('127.0.0.1', system.one) WITH TOTALS) LIMIT 1;

View File

@ -0,0 +1,6 @@
2
0 1
0 1
2
0 1
0 1

View File

@ -0,0 +1,6 @@
SET global_subqueries_method = 'push';
SELECT count() FROM remote('127.0.0.{1,2}', system.one) WHERE dummy GLOBAL IN (SELECT 0);
SELECT dummy, x FROM remote('127.0.0.{1,2}', system.one) GLOBAL ANY LEFT JOIN (SELECT 0 AS dummy, 1 AS x) USING dummy;
SET global_subqueries_method = 'pull';
SELECT count() FROM remote('127.0.0.{1,2}', system.one) WHERE dummy GLOBAL IN (SELECT 0);
SELECT dummy, x FROM remote('127.0.0.{1,2}', system.one) GLOBAL ANY LEFT JOIN (SELECT 0 AS dummy, 1 AS x) USING dummy;

View File

@ -0,0 +1,42 @@
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1

View File

@ -0,0 +1,51 @@
drop table if exists sequence_test;
create table sequence_test (time UInt32, data UInt8) engine=Memory;
insert into sequence_test values (0,0),(1,0),(2,0),(3,0),(4,1),(5,2),(6,0),(7,0),(8,0),(9,0),(10,1),(11,1);
select 1 = sequenceMatch('')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('.')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('.*')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?3)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceMatch('(?4)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)(?1)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)(?1)(?1)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceMatch('(?1)(?1)(?1)(?1)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)(?1)(?1)(?1)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)(?t>10)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceMatch('(?1)(?t>11)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)(?t<11)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)(?t<3)(?3)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?1)(?t<=2)(?3)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceMatch('(?1)(?t<2)(?3)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?2)(?t>=7)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceMatch('(?2)(?t>7)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceMatch('(?2)(?3)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select count() = sequenceCount('')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select count() = sequenceCount('.')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select count() = sequenceCount('.*')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 8 = sequenceCount('(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 3 = sequenceCount('(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceCount('(?3)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceCount('(?4)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 4 = sequenceCount('(?1)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 2 = sequenceCount('(?1)(?1)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 2 = sequenceCount('(?1)(?1)(?1)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceCount('(?1)(?1)(?1)(?1)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 2 = sequenceCount('(?1)(?1)(?1)(?1)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceCount('(?1)(?t>10)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceCount('(?1)(?t>11)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 2 = sequenceCount('(?1)(?t<11)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceCount('(?1)(?t<3)(?3)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceCount('(?1)(?t<=2)(?3)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceCount('(?1)(?t<2)(?3)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceCount('(?2)(?t>=7)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 0 = sequenceCount('(?2)(?t>7)(?2)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
select 1 = sequenceCount('(?2)(?3)(?1)')(toDateTime(time), data = 0, data = 1, data = 2, data = 3) from sequence_test;
drop table sequence_test;

View File

@ -0,0 +1,70 @@
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1
1 1 1

View File

@ -0,0 +1,104 @@
SET max_block_size = 1000;
DROP TABLE IF EXISTS test.numbers_10;
CREATE TABLE test.numbers_10 ENGINE = Memory AS SELECT * FROM system.numbers LIMIT 10000;
SET distributed_aggregation_memory_efficient = 0;
SET group_by_two_level_threshold = 1000;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 0;
SET group_by_two_level_threshold = 7;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 1000;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 7;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 1;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 1000;
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 1;
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{1,2}', test.numbers_10) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
DROP TABLE test.numbers_10;

View File

@ -0,0 +1,24 @@
0 2
1 2
2 2
3 2
4 2
5 2
6 2
7 2
8 2
9 2
0 200000
0 2
1 2
2 2
3 2
4 2
5 2
6 2
7 2
8 2
9 2
0 200000

View File

@ -0,0 +1,14 @@
DROP TABLE IF EXISTS test.numbers_100k_log;
CREATE TABLE test.numbers_100k_log ENGINE = Log AS SELECT * FROM system.numbers LIMIT 100000;
SELECT number, count() FROM remote('127.0.0.{1,2}', test.numbers_100k_log) GROUP BY number WITH TOTALS ORDER BY number LIMIT 10;
SET distributed_aggregation_memory_efficient = 1,
group_by_two_level_threshold = 1000,
group_by_overflow_mode = 'any',
max_rows_to_group_by = 1000,
totals_mode = 'after_having_auto';
SELECT number, count() FROM remote('127.0.0.{1,2}', test.numbers_100k_log) GROUP BY number WITH TOTALS ORDER BY number LIMIT 10;
DROP TABLE test.numbers_100k_log;