This commit is contained in:
Pavel Kartavyy 2015-08-20 15:12:15 +03:00
commit c9e318a2c4
146 changed files with 4128 additions and 1692 deletions

View File

@ -120,6 +120,10 @@ struct AggregateFunctionSequenceMatchData final
}
};
/// Max number of iterations to match the pattern against a sequence, exception thrown when exceeded
constexpr auto sequence_match_max_iterations = 1000000;
class AggregateFunctionSequenceMatch final : public IAggregateFunctionHelper<AggregateFunctionSequenceMatchData>
{
public:
@ -385,6 +389,7 @@ private:
return false;
};
std::size_t i = 0;
while (action_it != action_end && events_it != events_end)
{
// std::cout << "start_timestamp " << base_it->first << "; ";
@ -465,6 +470,12 @@ private:
"Unknown PatternActionType",
ErrorCodes::LOGICAL_ERROR
};
if (++i > sequence_match_max_iterations)
throw Exception{
"Pattern application proves too difficult, exceeding max iterations (" + toString(sequence_match_max_iterations) + ")",
ErrorCodes::TOO_SLOW
};
}
/// if there are some actions remaining

View File

@ -25,32 +25,7 @@
namespace DB
{
template <typename T> struct AggregateFunctionUniqTraits
{
static UInt64 hash(T x) { return x; }
};
template <> struct AggregateFunctionUniqTraits<Float32>
{
static UInt64 hash(Float32 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return res;
}
};
template <> struct AggregateFunctionUniqTraits<Float64>
{
static UInt64 hash(Float64 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return res;
}
};
/// uniq
struct AggregateFunctionUniqUniquesHashSetData
{
@ -60,6 +35,7 @@ struct AggregateFunctionUniqUniquesHashSetData
static String getName() { return "uniq"; }
};
/// uniqHLL12
template <typename T>
struct AggregateFunctionUniqHLL12Data
@ -79,6 +55,7 @@ struct AggregateFunctionUniqHLL12Data<String>
static String getName() { return "uniqHLL12"; }
};
/// uniqExact
template <typename T>
struct AggregateFunctionUniqExactData
@ -121,8 +98,8 @@ struct AggregateFunctionUniqExactData<String>
template <typename T>
struct AggregateFunctionUniqCombinedData
{
using Key = T;
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, DefaultHash<Key>, HashTableGrower<4> >, 16, 16, 19>;
using Key = UInt32;
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, HashTableGrower<> >, 16, 14, 17, TrivialHash>;
Set set;
static String getName() { return "uniqCombined"; }
@ -132,7 +109,7 @@ template <>
struct AggregateFunctionUniqCombinedData<String>
{
using Key = UInt64;
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, DefaultHash<Key>, HashTableGrower<4> >, 16, 16, 19>;
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, HashTableGrower<> >, 16, 14, 17, TrivialHash>;
Set set;
static String getName() { return "uniqCombined"; }
@ -140,75 +117,155 @@ struct AggregateFunctionUniqCombinedData<String>
namespace detail
{
/** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq.
* Используется для частичной специализации для добавления строк.
*/
template<typename T, typename Data>
struct OneAdder
/** Хэш-функция для uniqCombined.
*/
template<typename T, typename Enable = void>
struct CombinedCardinalityTraits
{
static UInt32 hash(T key)
{
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]));
}
};
return key;
}
};
template<typename Data>
struct OneAdder<String, Data>
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)
{
static void addOne(Data & data, const IColumn & column, size_t row_num)
{
/// Имейте ввиду, что вычисление приближённое.
StringRef value = column.getDataAt(row_num);
data.set.insert(CityHash64(value.data, value.size));
}
return intHash32<0>(static_cast<U>(key));
};
};
template<typename T>
struct OneAdder<T, AggregateFunctionUniqExactData<T> >
template<typename T>
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, UInt64>::value>::type>
{
static UInt32 hash(T key)
{
static void addOne(AggregateFunctionUniqExactData<T> & data, const IColumn & column, size_t row_num)
{
data.set.insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
return intHash32<0>(key);
};
};
template<>
struct OneAdder<String, AggregateFunctionUniqExactData<String> >
template<typename T>
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, Float64>::value>::type>
{
static UInt32 hash(T key)
{
static void addOne(AggregateFunctionUniqExactData<String> & data, const IColumn & column, size_t row_num)
{
StringRef value = column.getDataAt(row_num);
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&key), sizeof(key));
return intHash32<0>(res);
}
};
UInt128 key;
SipHash hash;
hash.update(value.data, value.size);
hash.get128(key.first, key.second);
data.set.insert(key);
}
};
template<typename T>
struct OneAdder<T, AggregateFunctionUniqCombinedData<T> >
template<typename T>
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, Float32>::value>::type>
{
static UInt32 hash(T key)
{
static void addOne(AggregateFunctionUniqCombinedData<T> & data, const IColumn & column, size_t row_num)
{
if (data.set.isMedium())
data.set.insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
else
data.set.insert(AggregateFunctionUniqTraits<T>::hash(static_cast<const ColumnVector<T> &>(column).getData()[row_num]));
}
};
UInt32 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&key), sizeof(key));
return res;
}
};
template<>
struct OneAdder<String, AggregateFunctionUniqCombinedData<String> >
/** Хэш-функция для uniq.
*/
template <typename T> struct AggregateFunctionUniqTraits
{
static UInt64 hash(T x) { return x; }
};
template <> struct AggregateFunctionUniqTraits<Float32>
{
static UInt64 hash(Float32 x)
{
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));
}
};
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return res;
}
};
template <> struct AggregateFunctionUniqTraits<Float64>
{
static UInt64 hash(Float64 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return res;
}
};
/** Структура для делегации работы по добавлению одного элемента в агрегатные функции 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 Data>
struct OneAdder<String, Data>
{
static void addOne(Data & data, const IColumn & column, size_t row_num)
{
/// Имейте ввиду, что вычисление приближённое.
StringRef value = column.getDataAt(row_num);
data.set.insert(CityHash64(value.data, value.size));
}
};
template<typename T>
struct OneAdder<T, AggregateFunctionUniqExactData<T> >
{
static void addOne(AggregateFunctionUniqExactData<T> & data, const IColumn & column, size_t row_num)
{
data.set.insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
};
template<>
struct OneAdder<String, AggregateFunctionUniqExactData<String> >
{
static void addOne(AggregateFunctionUniqExactData<String> & data, const IColumn & column, size_t row_num)
{
StringRef value = column.getDataAt(row_num);
UInt128 key;
SipHash hash;
hash.update(value.data, value.size);
hash.get128(key.first, key.second);
data.set.insert(key);
}
};
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,124 @@
#pragma once
#include <malloc.h>
#include <string.h>
#include <sys/mman.h>
#include <DB/Common/MemoryTracker.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
/** Отвечает за выделение/освобождение памяти. Используется, например, в PODArray, Arena.
* Интерфейс отличается от std::allocator
* - наличием метода realloc, который для больших кусков памяти использует mremap;
* - передачей размера в метод free;
* - наличием аргумента alignment;
*/
class Allocator
{
private:
/** См. комментарий в HashTableAllocator.h
*/
static constexpr size_t MMAP_THRESHOLD = 64 * (1 << 20);
static constexpr size_t HUGE_PAGE_SIZE = 2 * (1 << 20);
static constexpr size_t MMAP_MIN_ALIGNMENT = 4096;
static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
public:
/// Выделить кусок памяти.
void * alloc(size_t size, size_t alignment = 0)
{
if (current_memory_tracker)
current_memory_tracker->alloc(size);
void * buf;
if (size >= MMAP_THRESHOLD)
{
if (alignment > MMAP_MIN_ALIGNMENT)
throw DB::Exception("Too large alignment: more than page size.", DB::ErrorCodes::BAD_ARGUMENTS);
buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// См. комментарий в HashTableAllocator.h
if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE))
DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
}
else
{
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
buf = ::malloc(size);
if (nullptr == buf)
DB::throwFromErrno("Allocator: Cannot malloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
}
else
{
buf = nullptr;
int res = posix_memalign(&buf, alignment, size);
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
}
}
return buf;
}
/// Освободить память.
void free(void * buf, size_t size)
{
if (size >= MMAP_THRESHOLD)
{
if (0 != munmap(buf, size))
DB::throwFromErrno("Allocator: Cannot munmap.", DB::ErrorCodes::CANNOT_MUNMAP);
}
else
{
::free(buf);
}
if (current_memory_tracker)
current_memory_tracker->free(size);
}
/** Увеличить размер куска памяти.
* Содержимое старого куска памяти переезжает в начало нового.
* Положение куска памяти может измениться.
*/
void * realloc(void * buf, size_t old_size, size_t new_size, size_t alignment = 0)
{
if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD && alignment <= MALLOC_MIN_ALIGNMENT)
{
if (current_memory_tracker)
current_memory_tracker->realloc(old_size, new_size);
buf = ::realloc(buf, new_size);
if (nullptr == buf)
DB::throwFromErrno("Allocator: Cannot realloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
}
else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD)
{
if (current_memory_tracker)
current_memory_tracker->realloc(old_size, new_size);
buf = mremap(buf, old_size, new_size, MREMAP_MAYMOVE);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mremap.", DB::ErrorCodes::CANNOT_MREMAP);
}
else
{
void * new_buf = alloc(new_size, alignment);
memcpy(new_buf, buf, old_size);
free(buf, old_size);
buf = new_buf;
}
return buf;
}
};

View File

@ -6,7 +6,7 @@
#include <Poco/SharedPtr.h>
#include <Yandex/likely.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/MemoryTracker.h>
#include <DB/Common/Allocator.h>
namespace DB
@ -25,7 +25,7 @@ class Arena
{
private:
/// Непрерывный кусок памяти и указатель на свободное место в нём. Односвязный список.
struct Chunk : private std::allocator<char> /// empty base optimization
struct Chunk : private Allocator /// empty base optimization
{
char * begin;
char * pos;
@ -38,10 +38,7 @@ private:
ProfileEvents::increment(ProfileEvents::ArenaAllocChunks);
ProfileEvents::increment(ProfileEvents::ArenaAllocBytes, size_);
if (current_memory_tracker)
current_memory_tracker->alloc(size_);
begin = allocate(size_);
begin = reinterpret_cast<char *>(Allocator::alloc(size_));
pos = begin;
end = begin + size_;
prev = prev_;
@ -49,10 +46,7 @@ private:
~Chunk()
{
deallocate(begin, size());
if (current_memory_tracker)
current_memory_tracker->free(size());
Allocator::free(begin, size());
if (prev)
delete prev;

View File

@ -2,7 +2,8 @@
#include <DB/Common/HashTable/SmallTable.h>
#include <DB/Common/HashTable/HashSet.h>
#include <DB/Common/HyperLogLogWithSmallSetOptimization.h>
#include <statdaemons/HyperLogLogCounter.h>
#include <DB/Core/Defines.h>
namespace DB
@ -11,11 +12,11 @@ namespace DB
namespace details
{
enum class ContainerType { SMALL, MEDIUM, LARGE };
enum class ContainerType : UInt8 { SMALL = 1, MEDIUM = 2, LARGE = 3 };
ContainerType max(const ContainerType & lhs, const ContainerType & rhs)
static inline ContainerType max(const ContainerType & lhs, const ContainerType & rhs)
{
unsigned int res = std::max(static_cast<unsigned int>(lhs), static_cast<unsigned int>(rhs));
UInt8 res = std::max(static_cast<UInt8>(lhs), static_cast<UInt8>(rhs));
return static_cast<ContainerType>(res);
}
@ -25,38 +26,41 @@ ContainerType max(const ContainerType & lhs, const ContainerType & rhs)
* Для среднего - выделяется HashSet.
* Для большого - выделяется HyperLogLog.
*/
template <typename Key, typename HashContainer, UInt8 small_set_size_max, UInt8 medium_set_power2_max, UInt8 K>
template
<
typename Key,
typename HashContainer,
UInt8 small_set_size_max,
UInt8 medium_set_power2_max,
UInt8 K,
typename Hash = IntHash32<Key>,
typename DenominatorType = double
>
class CombinedCardinalityEstimator
{
public:
using Self = CombinedCardinalityEstimator<Key, HashContainer, small_set_size_max, medium_set_power2_max, K>;
using Self = CombinedCardinalityEstimator<Key, HashContainer, small_set_size_max, medium_set_power2_max, K, Hash, DenominatorType>;
private:
using Small = SmallSet<Key, small_set_size_max>;
using Medium = HashContainer;
using Large = HyperLogLogWithSmallSetOptimization<Key, small_set_size_max, K>;
using Large = HyperLogLogCounter<K, Hash, DenominatorType>;
public:
CombinedCardinalityEstimator()
{
setContainerType(details::ContainerType::SMALL);
}
~CombinedCardinalityEstimator()
{
if (container_type == details::ContainerType::MEDIUM)
{
delete medium;
if (current_memory_tracker)
current_memory_tracker->free(sizeof(medium));
}
else if (container_type == details::ContainerType::LARGE)
{
delete large;
if (current_memory_tracker)
current_memory_tracker->free(sizeof(large));
}
destroy();
}
void insert(Key value)
{
auto container_type = getContainerType();
if (container_type == details::ContainerType::SMALL)
{
if (small.find(value) == small.end())
@ -66,41 +70,43 @@ public:
else
{
toMedium();
medium->insert(value);
getContainer<Medium>().insert(value);
}
}
}
else if (container_type == details::ContainerType::MEDIUM)
{
if (medium->size() < medium_set_size_max)
medium->insert(value);
auto & container = getContainer<Medium>();
if (container.size() < medium_set_size_max)
container.insert(value);
else
{
toLarge();
large->insert(value);
getContainer<Large>().insert(value);
}
}
else if (container_type == details::ContainerType::LARGE)
large->insert(value);
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
getContainer<Large>().insert(value);
}
UInt32 size() const
{
auto container_type = getContainerType();
if (container_type == details::ContainerType::SMALL)
return small.size();
else if (container_type == details::ContainerType::MEDIUM)
return medium->size();
return getContainer<Medium>().size();
else if (container_type == details::ContainerType::LARGE)
return large->size();
return getContainer<Large>().size();
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
void merge(const Self & rhs)
{
details::ContainerType max_container_type = details::max(container_type, rhs.container_type);
auto container_type = getContainerType();
auto max_container_type = details::max(container_type, rhs.getContainerType());
if (container_type != max_container_type)
{
@ -110,41 +116,18 @@ public:
toLarge();
}
if (container_type == details::ContainerType::SMALL)
if (rhs.getContainerType() == details::ContainerType::SMALL)
{
for (const auto & x : rhs.small)
insert(x);
}
else if (container_type == details::ContainerType::MEDIUM)
else if (rhs.getContainerType() == details::ContainerType::MEDIUM)
{
if (rhs.container_type == details::ContainerType::SMALL)
{
for (const auto & x : rhs.small)
insert(x);
}
else if (rhs.container_type == details::ContainerType::MEDIUM)
{
for (const auto & x : *rhs.medium)
insert(x);
}
for (const auto & x : rhs.getContainer<Medium>())
insert(x);
}
else if (container_type == details::ContainerType::LARGE)
{
if (rhs.container_type == details::ContainerType::SMALL)
{
for (const auto & x : rhs.small)
insert(x);
}
else if (rhs.container_type == details::ContainerType::MEDIUM)
{
for (const auto & x : *rhs.medium)
insert(x);
}
else if (rhs.container_type == details::ContainerType::LARGE)
large->merge(*rhs.large);
}
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
else if (rhs.getContainerType() == details::ContainerType::LARGE)
getContainer<Large>().merge(rhs.getContainer<Large>());
}
/// Можно вызывать только для пустого объекта.
@ -152,79 +135,95 @@ public:
{
UInt8 v;
readBinary(v, in);
details::ContainerType t = static_cast<details::ContainerType>(v);
auto container_type = static_cast<details::ContainerType>(v);
if (t == details::ContainerType::SMALL)
if (container_type == details::ContainerType::SMALL)
small.read(in);
else if (t == details::ContainerType::MEDIUM)
else if (container_type == details::ContainerType::MEDIUM)
{
toMedium();
medium->read(in);
getContainer<Medium>().read(in);
}
else if (t == details::ContainerType::LARGE)
else if (container_type == details::ContainerType::LARGE)
{
toLarge();
large->read(in);
getContainer<Large>().read(in);
}
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
void readAndMerge(DB::ReadBuffer & in)
{
Self other;
other.read(in);
merge(other);
auto container_type = getContainerType();
UInt8 v;
readBinary(v, in);
auto rhs_container_type = static_cast<details::ContainerType>(v);
auto max_container_type = details::max(container_type, rhs_container_type);
if (container_type != max_container_type)
{
if (max_container_type == details::ContainerType::MEDIUM)
toMedium();
else if (max_container_type == details::ContainerType::LARGE)
toLarge();
}
if (rhs_container_type == details::ContainerType::SMALL)
{
typename Small::Reader reader(in);
while (reader.next())
insert(reader.get());
}
else if (rhs_container_type == details::ContainerType::MEDIUM)
{
typename Medium::Reader reader(in);
while (reader.next())
insert(reader.get());
}
else if (rhs_container_type == details::ContainerType::LARGE)
getContainer<Large>().readAndMerge(in);
}
void write(DB::WriteBuffer & out) const
{
UInt8 v = static_cast<UInt8>(container_type);
writeBinary(v, out);
auto container_type = getContainerType();
writeBinary(static_cast<UInt8>(container_type), out);
if (container_type == details::ContainerType::SMALL)
small.write(out);
else if (container_type == details::ContainerType::MEDIUM)
medium->write(out);
getContainer<Medium>().write(out);
else if (container_type == details::ContainerType::LARGE)
large->write(out);
else
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
}
bool isMedium() const
{
return container_type == details::ContainerType::MEDIUM;
getContainer<Large>().write(out);
}
private:
void toMedium()
{
if (container_type != details::ContainerType::SMALL)
if (getContainerType() != details::ContainerType::SMALL)
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
if (current_memory_tracker)
current_memory_tracker->alloc(sizeof(medium));
Medium * tmp_medium = new Medium;
auto tmp_medium = std::make_unique<Medium>();
for (const auto & x : small)
tmp_medium->insert(x);
medium = tmp_medium;
medium = tmp_medium.release();
setContainerType(details::ContainerType::MEDIUM);
container_type = details::ContainerType::MEDIUM;
if (current_memory_tracker)
current_memory_tracker->alloc(sizeof(medium));
}
void toLarge()
{
auto container_type = getContainerType();
if ((container_type != details::ContainerType::SMALL) && (container_type != details::ContainerType::MEDIUM))
throw Poco::Exception("Internal error", ErrorCodes::LOGICAL_ERROR);
if (current_memory_tracker)
current_memory_tracker->alloc(sizeof(large));
Large * tmp_large = new Large;
auto tmp_large = std::make_unique<Large>();
if (container_type == details::ContainerType::SMALL)
{
@ -233,11 +232,25 @@ private:
}
else if (container_type == details::ContainerType::MEDIUM)
{
for (const auto & x : *medium)
for (const auto & x : getContainer<Medium>())
tmp_large->insert(x);
destroy();
}
large = tmp_large;
large = tmp_large.release();
setContainerType(details::ContainerType::LARGE);
if (current_memory_tracker)
current_memory_tracker->alloc(sizeof(large));
}
void NO_INLINE destroy()
{
auto container_type = getContainerType();
clearContainerType();
if (container_type == details::ContainerType::MEDIUM)
{
@ -247,16 +260,54 @@ private:
if (current_memory_tracker)
current_memory_tracker->free(sizeof(medium));
}
else if (container_type == details::ContainerType::LARGE)
{
delete large;
large = nullptr;
container_type = details::ContainerType::LARGE;
if (current_memory_tracker)
current_memory_tracker->free(sizeof(large));
}
}
template<typename T>
inline T & getContainer()
{
return *reinterpret_cast<T *>(address & mask);
}
template<typename T>
inline const T & getContainer() const
{
return *reinterpret_cast<T *>(address & mask);
}
void setContainerType(details::ContainerType t)
{
address &= mask;
address |= static_cast<UInt8>(t);
}
inline details::ContainerType getContainerType() const
{
return static_cast<details::ContainerType>(address & ~mask);
}
void clearContainerType()
{
address &= mask;
}
private:
Small small;
Medium * medium = nullptr;
Large * large = nullptr;
const UInt32 medium_set_size_max = 1UL << medium_set_power2_max;
details::ContainerType container_type = details::ContainerType::SMALL;
union
{
Medium * medium;
Large * large;
UInt64 address = 0;
};
static const UInt64 mask = 0xFFFFFFFFFFFFFFFC;
static const UInt32 medium_set_size_max = 1UL << medium_set_power2_max;
};
}

View File

@ -16,12 +16,11 @@ namespace DB
{
/// хранит размеры всех столбцов, и может проверять не побились ли столбцы
template <class Storage>
class FileChecker
{
public:
FileChecker(const std::string &file_info_path_, Storage & storage_) :
files_info_path(file_info_path_), files_info(), storage(storage_), log(&Logger::get("FileChecker"))
FileChecker(const std::string & file_info_path_) :
files_info_path(file_info_path_), files_info(), log(&Logger::get("FileChecker"))
{
Poco::Path path(files_info_path);
tmp_files_info_path = path.parent().toString() + "tmp_" + path.getFileName();
@ -107,7 +106,6 @@ private:
using PropertyTree = boost::property_tree::ptree;
PropertyTree files_info;
Storage & storage;
Logger * log;
};
}

View File

@ -251,6 +251,7 @@ class HashTable :
protected:
friend class const_iterator;
friend class iterator;
friend class Reader;
template <typename, typename, typename, typename, typename, typename, size_t>
friend class TwoLevelHashTable;
@ -429,6 +430,51 @@ public:
free();
}
class Reader final : private Cell::State
{
public:
Reader(DB::ReadBuffer & in_)
: in(in_)
{
}
Reader(const Reader &) = delete;
Reader & operator=(const Reader &) = delete;
bool next()
{
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;
return true;
}
inline const value_type & get() const
{
if ((read_count == 0) || is_eof)
throw DB::Exception("No available data", DB::ErrorCodes::NO_AVAILABLE_DATA);
return cell.getValue();
}
private:
DB::ReadBuffer in;
Cell cell;
size_t read_count = 0;
size_t size;
bool is_eof = false;
};
class iterator
{
@ -757,7 +803,7 @@ public:
{
Cell x;
x.read(rb);
insert(Cell::getKey(x.getValue()));
insert(x);
}
}
@ -781,7 +827,7 @@ public:
Cell x;
DB::assertString(",", rb);
x.readText(rb);
insert(Cell::getKey(x.getValue()));
insert(x);
}
}

View File

@ -19,6 +19,7 @@
/** Общая часть разных хэш-таблиц, отвечающая за выделение/освобождение памяти.
* Отличается от Allocator тем, что зануляет память.
* Используется в качестве параметра шаблона (есть несколько реализаций с таким же интерфейсом).
*/
class HashTableAllocator
@ -33,9 +34,9 @@ private:
* Рассчитываем, что набор операций mmap/что-то сделать/mremap может выполняться всего лишь около 1000 раз в секунду.
*
* PS. Также это требуется, потому что tcmalloc не может выделить кусок памяти больше 16 GB.
* NOTE Можно попробовать MAP_HUGETLB, но придётся самостоятельно управлять количеством доступных страниц.
*/
static constexpr size_t MMAP_THRESHOLD = 64 * (1 << 20);
static constexpr size_t HUGE_PAGE_SIZE = 2 * (1 << 20);
public:
/// Выделить кусок памяти и заполнить его нулями.
@ -52,6 +53,14 @@ public:
if (MAP_FAILED == buf)
DB::throwFromErrno("HashTableAllocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/** Использование huge pages позволяет увеличить производительность более чем в три раза
* в запросе SELECT number % 1000000 AS k, count() FROM system.numbers GROUP BY k,
* (хэш-таблица на 1 000 000 элементов)
* и примерно на 15% в случае хэш-таблицы на 100 000 000 элементов.
*/
if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE))
DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// Заполнение нулями не нужно - mmap сам это делает.
}
else
@ -108,6 +117,10 @@ public:
if (MAP_FAILED == buf)
DB::throwFromErrno("HashTableAllocator: Cannot mremap.", DB::ErrorCodes::CANNOT_MREMAP);
/** Здесь не получается сделать madvise с MADV_HUGEPAGE.
* Похоже, что при mremap, huge pages сами расширяются на новую область.
*/
/// Заполнение нулями не нужно.
}
else

View File

@ -27,6 +27,7 @@ class SmallTable :
protected:
friend class const_iterator;
friend class iterator;
friend class Reader;
typedef SmallTable<Key, Cell, capacity> Self;
typedef Cell cell_type;
@ -66,6 +67,55 @@ public:
typedef typename Cell::value_type value_type;
class Reader final : private Cell::State
{
public:
Reader(DB::ReadBuffer & in_)
: in(in_)
{
}
Reader(const Reader &) = delete;
Reader & operator=(const Reader &) = delete;
bool next()
{
if (read_count == size)
{
is_eof = true;
return false;
}
else if (read_count == 0)
{
Cell::State::read(in);
DB::readVarUInt(size, in);
if (size > capacity)
throw DB::Exception("Illegal size");
}
cell.read(in);
++read_count;
return true;
}
inline const value_type & get() const
{
if ((read_count == 0) || is_eof)
throw DB::Exception("No available data", DB::ErrorCodes::NO_AVAILABLE_DATA);
return cell.getValue();
}
private:
DB::ReadBuffer in;
Cell cell;
size_t read_count = 0;
size_t size;
bool is_eof = false;
};
class iterator
{
Self * container;

View File

@ -16,7 +16,7 @@ template <
UInt8 small_set_size,
UInt8 K,
typename Hash = IntHash32<Key>,
typename DenominatorType = float>
typename DenominatorType = double>
class HyperLogLogWithSmallSetOptimization
{
private:
@ -114,10 +114,20 @@ public:
void readAndMerge(DB::ReadBuffer & in)
{
/// Немного не оптимально.
HyperLogLogWithSmallSetOptimization other;
other.read(in);
merge(other);
bool is_rhs_large;
readBinary(is_rhs_large, in);
if (!isLarge() && is_rhs_large)
toLarge();
if (!is_rhs_large)
{
typename Small::Reader reader(in);
while (reader.next())
insert(reader.get());
}
else
large->readAndMerge(in);
}
void write(DB::WriteBuffer & out) const

View File

@ -1,7 +1,6 @@
#pragma once
#include <string.h>
#include <malloc.h>
#include <cstddef>
#include <algorithm>
#include <memory>
@ -12,7 +11,7 @@
#include <Yandex/likely.h>
#include <Yandex/strong_typedef.h>
#include <DB/Common/MemoryTracker.h>
#include <DB/Common/Allocator.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
@ -32,28 +31,18 @@ namespace DB
* Конструктор по-умолчанию создаёт пустой объект, который не выделяет память.
* Затем выделяется память минимум под POD_ARRAY_INITIAL_SIZE элементов.
*
* При первом выделении памяти использует std::allocator.
* В реализации из libstdc++ он кэширует куски памяти несколько больше, чем обычный malloc.
*
* При изменении размера, использует realloc, который может (но не обязан) использовать mremap для больших кусков памяти.
* По факту, mremap используется при использовании аллокатора из glibc, но не используется, например, в tcmalloc.
*
* Если вставлять элементы push_back-ом, не делая reserve, то PODArray примерно в 2.5 раза быстрее std::vector.
*/
#define POD_ARRAY_INITIAL_SIZE 4096UL
template <typename T>
class PODArray : private boost::noncopyable, private std::allocator<char> /// empty base optimization
class PODArray : private boost::noncopyable, private Allocator /// empty base optimization
{
private:
typedef std::allocator<char> Allocator;
char * c_start;
char * c_end;
char * c_end_of_storage;
bool use_libc_realloc = false;
T * t_start() { return reinterpret_cast<T *>(c_start); }
T * t_end() { return reinterpret_cast<T *>(c_end); }
T * t_end_of_storage() { return reinterpret_cast<T *>(c_end_of_storage); }
@ -90,10 +79,7 @@ private:
size_t bytes_to_alloc = to_size(n);
if (current_memory_tracker)
current_memory_tracker->alloc(bytes_to_alloc);
c_start = c_end = Allocator::allocate(bytes_to_alloc);
c_start = c_end = reinterpret_cast<char *>(Allocator::alloc(bytes_to_alloc));
c_end_of_storage = c_start + bytes_to_alloc;
}
@ -102,13 +88,7 @@ private:
if (c_start == nullptr)
return;
if (use_libc_realloc)
::free(c_start);
else
Allocator::deallocate(c_start, storage_size());
if (current_memory_tracker)
current_memory_tracker->free(storage_size());
Allocator::free(c_start, storage_size());
}
void realloc(size_t n)
@ -122,38 +102,10 @@ private:
ptrdiff_t end_diff = c_end - c_start;
size_t bytes_to_alloc = to_size(n);
char * old_c_start = c_start;
char * old_c_end_of_storage = c_end_of_storage;
if (current_memory_tracker)
current_memory_tracker->realloc(storage_size(), bytes_to_alloc);
if (use_libc_realloc)
{
auto new_c_start = reinterpret_cast<char *>(::realloc(c_start, bytes_to_alloc));
if (nullptr == new_c_start)
throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
c_start = new_c_start;
}
else
{
auto new_c_start = reinterpret_cast<char *>(malloc(bytes_to_alloc));
if (nullptr == new_c_start)
throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
c_start = new_c_start;
memcpy(c_start, old_c_start, std::min(bytes_to_alloc, static_cast<size_t>(end_diff)));
Allocator::deallocate(old_c_start, old_c_end_of_storage - old_c_start);
}
c_start = reinterpret_cast<char *>(Allocator::realloc(c_start, storage_size(), bytes_to_alloc));
c_end = c_start + end_diff;
c_end_of_storage = c_start + bytes_to_alloc;
use_libc_realloc = true;
}
public:
@ -187,7 +139,6 @@ public:
std::swap(c_start, other.c_start);
std::swap(c_end, other.c_end);
std::swap(c_end_of_storage, other.c_end_of_storage);
std::swap(use_libc_realloc, other.use_libc_realloc);
return *this;
}

View File

@ -283,6 +283,9 @@ namespace ErrorCodes
INDEX_NOT_USED = 277,
LEADERSHIP_LOST = 278,
ALL_CONNECTION_TRIES_FAILED = 279,
NO_AVAILABLE_DATA = 280,
DICTIONARY_IS_EMPTY = 281,
INCORRECT_INDEX = 282,
KEEPER_EXCEPTION = 999,
POCO_EXCEPTION = 1000,

View File

@ -28,8 +28,8 @@ ExceptionPtr cloneCurrentException();
/** Попробовать записать исключение в лог (и забыть про него).
* Можно использовать в деструкторах в блоке catch (...).
*/
void tryLogCurrentException(const char * log_name);
void tryLogCurrentException(Poco::Logger * logger);
void tryLogCurrentException(const char * log_name, const std::string & start_of_message = "");
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = "");
std::string getCurrentExceptionMessage(bool with_stacktrace);

View File

@ -14,10 +14,10 @@ class FormatFactory
{
public:
BlockInputStreamPtr getInput(const String & name, ReadBuffer & buf,
Block & sample, size_t max_block_size) const;
const Block & sample, size_t max_block_size) const;
BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf,
Block & sample) const;
const Block & sample) const;
};
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <tuple>
#include <DB/Core/Types.h>
#include <DB/IO/WriteHelpers.h>
namespace DB
{
/** Засечка - позиция в сжатом файле. Сжатый файл состоит из уложенных подряд сжатых блоков.
* Засечка представляют собой пару - смещение в файле до начала сжатого блока, смещение в разжатом блоке до начала данных.
*/
struct MarkInCompressedFile
{
size_t offset_in_compressed_file;
size_t offset_in_decompressed_block;
bool operator==(const MarkInCompressedFile & rhs) const
{
return std::tie(offset_in_compressed_file, offset_in_decompressed_block)
== std::tie(rhs.offset_in_compressed_file, rhs.offset_in_decompressed_block);
}
bool operator!=(const MarkInCompressedFile & rhs) const
{
return !(*this == rhs);
}
String toString() const
{
return "(" + DB::toString(offset_in_compressed_file) + "," + DB::toString(offset_in_decompressed_block) + ")";
}
};
using MarksInCompressedFile = std::vector<MarkInCompressedFile>;
}

View File

@ -0,0 +1,110 @@
#pragma once
#include <DB/Interpreters/Aggregator.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
/** Доагрегирует потоки блоков, держа в оперативной памяти только по одному блоку из каждого потока.
* Это экономит оперативку в случае использования двухуровневой агрегации, где в каждом потоке будет до 256 блоков с частями результата.
*
* Агрегатные функции в блоках не должны быть финализированы, чтобы их состояния можно было объединить.
*/
class MergingAggregatedMemoryEfficientBlockInputStream : public IProfilingBlockInputStream
{
public:
MergingAggregatedMemoryEfficientBlockInputStream(BlockInputStreams inputs_, const Names & keys_names_,
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_)
: aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0),
final(final_)
{
children = inputs_;
current_blocks.resize(children.size());
overflow_blocks.resize(children.size());
is_exhausted.resize(children.size());
}
String getName() const override { return "MergingAggregatedMemorySavvy"; }
String getID() const override
{
std::stringstream res;
res << "MergingAggregatedMemorySavvy(" << aggregator.getID();
for (size_t i = 0, size = children.size(); i < size; ++i)
res << ", " << children.back()->getID();
res << ")";
return res.str();
}
protected:
Block readImpl() override
{
/// Если child - RemoteBlockInputStream, то отправляет запрос на все удалённые серверы, инициируя вычисления.
if (current_bucket_num == -1)
for (auto & child : children)
child->readPrefix();
/// Всё прочитали.
if (current_bucket_num > 255)
return {};
/// Читаем следующие блоки для current_bucket_num
for (size_t i = 0, size = children.size(); i < size; ++i)
{
while (!is_exhausted[i] && (!current_blocks[i] || current_blocks[i].info.bucket_num < current_bucket_num))
{
current_blocks[i] = children[i]->read();
if (!current_blocks[i])
{
is_exhausted[i] = true;
}
else if (current_blocks[i].info.is_overflows)
{
overflow_blocks[i].swap(current_blocks[i]);
}
}
}
/// Может быть, нет блоков для current_bucket_num, а все блоки имеют больший bucket_num.
Int32 min_bucket_num = 256;
for (size_t i = 0, size = children.size(); i < size; ++i)
if (!is_exhausted[i] && current_blocks[i].info.bucket_num < min_bucket_num)
min_bucket_num = current_blocks[i].info.bucket_num;
current_bucket_num = min_bucket_num;
/// Все потоки исчерпаны.
if (current_bucket_num > 255)
return {}; /// TODO overflow_blocks.
/// TODO Если есть single_level и two_level блоки.
/// Объединяем все блоки с current_bucket_num.
BlocksList blocks_to_merge;
for (size_t i = 0, size = children.size(); i < size; ++i)
if (current_blocks[i].info.bucket_num == current_bucket_num)
blocks_to_merge.emplace_back(std::move(current_blocks[i]));
Block res = aggregator.mergeBlocks(blocks_to_merge, final);
++current_bucket_num;
return res;
}
private:
Aggregator aggregator;
bool final;
Int32 current_bucket_num = -1;
std::vector<Block> current_blocks;
std::vector<UInt8> is_exhausted;
std::vector<Block> overflow_blocks;
};
}

View File

@ -1,22 +1,75 @@
#pragma once
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/DataStreams/MarkInCompressedFile.h>
namespace DB
{
class CompressedReadBufferFromFile;
/** Формат Native может содержать отдельно расположенный индекс,
* который позволяет понять, где какой столбец расположен,
* и пропускать ненужные столбцы.
*/
/** Позиция одного кусочка одного столбца. */
struct IndexOfOneColumnForNativeFormat
{
String name;
String type;
MarkInCompressedFile location;
};
/** Индекс для блока данных. */
struct IndexOfBlockForNativeFormat
{
using Columns = std::vector<IndexOfOneColumnForNativeFormat>;
size_t num_columns;
size_t num_rows;
Columns columns;
};
/** Весь индекс. */
struct IndexForNativeFormat
{
using Blocks = std::vector<IndexOfBlockForNativeFormat>;
Blocks blocks;
IndexForNativeFormat() {}
IndexForNativeFormat(ReadBuffer & istr, const NameSet & required_columns)
{
read(istr, required_columns);
}
/// Прочитать индекс, только для нужных столбцов.
void read(ReadBuffer & istr, const NameSet & required_columns);
};
/** Десериализует поток блоков из родного бинарного формата (с именами и типами столбцов).
* Предназначено для взаимодействия между серверами.
*
* Также может использоваться для хранения данных на диске.
* В этом случае, может использовать индекс.
*/
class NativeBlockInputStream : public IProfilingBlockInputStream
{
public:
/** В случае указания ненулевой server_revision, может ожидаться и считываться дополнительная информация о блоке,
* в зависимости от поддерживаемой для указанной ревизии.
*
* index - не обязательный параметр. Если задан, то будут читаться только указанные в индексе кусочки столбцов.
*/
NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_ = 0)
: istr(istr_), server_revision(server_revision_) {}
NativeBlockInputStream(
ReadBuffer & istr_, UInt64 server_revision_ = 0,
bool use_index_ = false,
IndexForNativeFormat::Blocks::const_iterator index_block_it_ = IndexForNativeFormat::Blocks::const_iterator{},
IndexForNativeFormat::Blocks::const_iterator index_block_end_ = IndexForNativeFormat::Blocks::const_iterator{});
String getName() const override { return "Native"; }
@ -35,6 +88,14 @@ protected:
private:
ReadBuffer & istr;
UInt64 server_revision;
bool use_index;
IndexForNativeFormat::Blocks::const_iterator index_block_it;
IndexForNativeFormat::Blocks::const_iterator index_block_end;
IndexOfBlockForNativeFormat::Columns::const_iterator index_column_it;
/// Если задан индекс, то istr должен быть CompressedReadBufferFromFile.
CompressedReadBufferFromFile * istr_concrete;
};
}

View File

@ -6,8 +6,14 @@
namespace DB
{
class WriteBuffer;
class CompressedWriteBuffer;
/** Сериализует поток блоков в родном бинарном формате (с именами и типами столбцов).
* Предназначено для взаимодействия между серверами.
*
* Может быть указан поток для записи индекса. Индекс содержит смещения до каждого кусочка каждого столбца.
*/
class NativeBlockOutputStream : public IBlockOutputStream
{
@ -15,8 +21,9 @@ public:
/** В случае указания ненулевой client_revision, может записываться дополнительная информация о блоке,
* в зависимости от поддерживаемой для указанной ревизии.
*/
NativeBlockOutputStream(WriteBuffer & ostr_, UInt64 client_revision_ = 0)
: ostr(ostr_), client_revision(client_revision_) {}
NativeBlockOutputStream(
WriteBuffer & ostr_, UInt64 client_revision_ = 0,
WriteBuffer * index_ostr_ = nullptr);
void write(const Block & block) override;
void flush() override { ostr.next(); }
@ -26,6 +33,10 @@ public:
private:
WriteBuffer & ostr;
UInt64 client_revision;
WriteBuffer * index_ostr;
/// Если требуется записывать индекс, то ostr обязан быть CompressedWriteBuffer.
CompressedWriteBuffer * ostr_concrete = nullptr;
};
}

View File

@ -84,6 +84,16 @@ public:
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
{
std::lock_guard<std::mutex> lock(external_tables_mutex);
/// Останавливаем отправку внешних данных.
for (auto & vec : external_tables_data)
for (auto & elem : vec)
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(elem.first.get()))
stream->cancel();
}
if (!isQueryPending() || hasThrownException())
return;
@ -101,52 +111,54 @@ public:
parallel_replicas->disconnect();
}
/// Отправляет запрос (инициирует вычисления) раньше, чем read.
void readPrefix() override
{
if (!sent_query)
sendQuery();
}
protected:
/// Отправить на удаленные реплики все временные таблицы
void sendExternalTables()
{
size_t count = parallel_replicas->size();
std::vector<ExternalTablesData> instances;
instances.reserve(count);
for (size_t i = 0; i < count; ++i)
{
ExternalTablesData res;
for (const auto & table : external_tables)
std::lock_guard<std::mutex> lock(external_tables_mutex);
external_tables_data.reserve(count);
for (size_t i = 0; i < count; ++i)
{
StoragePtr cur = table.second;
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings,
stage, DEFAULT_BLOCK_SIZE, 1);
if (input.size() == 0)
res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), table.first));
else
res.push_back(std::make_pair(input[0], table.first));
ExternalTablesData res;
for (const auto & table : external_tables)
{
StoragePtr cur = table.second;
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings,
stage, DEFAULT_BLOCK_SIZE, 1);
if (input.size() == 0)
res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), table.first));
else
res.push_back(std::make_pair(input[0], table.first));
}
external_tables_data.push_back(std::move(res));
}
instances.push_back(std::move(res));
}
parallel_replicas->sendExternalTablesData(instances);
parallel_replicas->sendExternalTablesData(external_tables_data);
}
Block readImpl() override
{
if (!sent_query)
{
createParallelReplicas();
sendQuery();
if (settings.skip_unavailable_shards && 0 == parallel_replicas->size())
return Block();
established = true;
parallel_replicas->sendQuery(query, "", stage, true);
established = false;
sent_query = true;
sendExternalTables();
return {};
}
while (true)
@ -267,6 +279,23 @@ protected:
}
private:
void sendQuery()
{
createParallelReplicas();
if (settings.skip_unavailable_shards && 0 == parallel_replicas->size())
return;
established = true;
parallel_replicas->sendQuery(query, "", stage, true);
established = false;
sent_query = true;
sendExternalTables();
}
/// ITable::read requires a Context, therefore we should create one if the user can't supply it
static Context & getDefaultContext()
{
@ -302,6 +331,10 @@ private:
QueryProcessingStage::Enum stage;
Context context;
/// Потоки для чтения из временных таблиц - для последующей отправки данных на удалённые серверы для GLOBAL-подзапросов.
std::vector<ExternalTablesData> external_tables_data;
std::mutex external_tables_mutex;
/// Установили соединения с репликами, но ещё не отправили запрос.
std::atomic<bool> established { false };

View File

@ -620,9 +620,9 @@ private:
mutable std::mt19937_64 rnd_engine{getSeed()};
mutable std::size_t bytes_allocated = 0;
mutable std::atomic<std::size_t> element_count{};
mutable std::atomic<std::size_t> hit_count{};
mutable std::atomic<std::size_t> query_count{};
mutable std::atomic<std::size_t> element_count{0};
mutable std::atomic<std::size_t> hit_count{0};
mutable std::atomic<std::size_t> query_count{0};
const std::chrono::time_point<std::chrono::system_clock> creation_time = std::chrono::system_clock::now();
};

View File

@ -20,8 +20,7 @@ class FileDictionarySource final : public IDictionarySource
public:
FileDictionarySource(const std::string & filename, const std::string & format, Block & sample_block,
const Context & context)
: filename{filename}, format{format}, sample_block{sample_block}, context(context),
last_modification{getLastModification()}
: filename{filename}, format{format}, sample_block{sample_block}, context(context)
{}
FileDictionarySource(const FileDictionarySource & other)

View File

@ -20,9 +20,10 @@ class FlatDictionary final : public IDictionary
{
public:
FlatDictionary(const std::string & name, const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime)
DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, bool require_nonempty)
: name{name}, dict_struct(dict_struct),
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime)
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty)
{
createAttributes();
@ -40,7 +41,7 @@ public:
}
FlatDictionary(const FlatDictionary & other)
: FlatDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime}
: FlatDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
{}
std::exception_ptr getCreationException() const override { return creation_exception; }
@ -198,6 +199,9 @@ private:
}
stream->readSuffix();
if (require_nonempty && 0 == element_count)
throw Exception("Dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY);
}
template <typename T>
@ -348,6 +352,7 @@ private:
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const bool require_nonempty;
std::map<std::string, std::size_t> attribute_index_by_name;
std::vector<attribute_t> attributes;
@ -356,7 +361,7 @@ private:
std::size_t bytes_allocated = 0;
std::size_t element_count = 0;
std::size_t bucket_count = 0;
mutable std::atomic<std::size_t> query_count;
mutable std::atomic<std::size_t> query_count{0};
std::chrono::time_point<std::chrono::system_clock> creation_time;

View File

@ -18,9 +18,10 @@ class HashedDictionary final : public IDictionary
{
public:
HashedDictionary(const std::string & name, const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime)
DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, bool require_nonempty)
: name{name}, dict_struct(dict_struct),
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime)
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty)
{
createAttributes();
@ -38,7 +39,7 @@ public:
}
HashedDictionary(const HashedDictionary & other)
: HashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime}
: HashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
{}
std::exception_ptr getCreationException() const override { return creation_exception; }
@ -196,6 +197,9 @@ private:
}
stream->readSuffix();
if (require_nonempty && 0 == element_count)
throw Exception("Dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY);
}
template <typename T>
@ -334,6 +338,7 @@ private:
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const bool require_nonempty;
std::map<std::string, std::size_t> attribute_index_by_name;
std::vector<attribute_t> attributes;
@ -342,7 +347,7 @@ private:
std::size_t bytes_allocated = 0;
std::size_t element_count = 0;
std::size_t bucket_count = 0;
mutable std::atomic<std::size_t> query_count{};
mutable std::atomic<std::size_t> query_count{0};
std::chrono::time_point<std::chrono::system_clock> creation_time;

View File

@ -23,10 +23,10 @@ public:
db{config.getString(config_prefix + ".db", "")},
table{config.getString(config_prefix + ".table")},
where{config.getString(config_prefix + ".where", "")},
dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)},
sample_block{sample_block},
pool{config, config_prefix},
load_all_query{composeLoadAllQuery()},
last_modification{getLastModification()}
load_all_query{composeLoadAllQuery()}
{}
/// copy-constructor is provided in order to support cloneability
@ -35,6 +35,7 @@ public:
db{other.db},
table{other.table},
where{other.where},
dont_check_update_time{other.dont_check_update_time},
sample_block{other.sample_block},
pool{other.pool},
load_all_query{other.load_all_query}, last_modification{other.last_modification}
@ -43,18 +44,27 @@ public:
BlockInputStreamPtr loadAll() override
{
last_modification = getLastModification();
LOG_TRACE(log, load_all_query);
return new MySQLBlockInputStream{pool.Get(), load_all_query, sample_block, max_block_size};
}
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
{
last_modification = getLastModification();
const auto query = composeLoadIdsQuery(ids);
/// Здесь не логгируем и не обновляем время модификации, так как запрос может быть большим, и часто задаваться.
const auto query = composeLoadIdsQuery(ids);
return new MySQLBlockInputStream{pool.Get(), query, sample_block, max_block_size};
}
bool isModified() const override { return getLastModification() > last_modification; }
bool isModified() const override
{
if (dont_check_update_time)
return true;
return getLastModification() > last_modification;
}
bool supportsSelectiveLoad() const override { return true; }
DictionarySourcePtr clone() const override { return std::make_unique<MySQLDictionarySource>(*this); }
@ -65,27 +75,47 @@ public:
}
private:
Logger * log = &Logger::get("MySQLDictionarySource");
mysqlxx::DateTime getLastModification() const
{
const auto Update_time_idx = 12;
mysqlxx::DateTime update_time{std::time(nullptr)};
if (dont_check_update_time)
return update_time;
try
{
auto connection = pool.Get();
auto query = connection->query("SHOW TABLE STATUS LIKE '%" + strconvert::escaped_for_like(table) + "%';");
auto query = connection->query("SHOW TABLE STATUS LIKE '" + strconvert::escaped_for_like(table) + "'");
LOG_TRACE(log, query.str());
auto result = query.use();
size_t fetched_rows = 0;
if (auto row = result.fetch())
{
const auto & update_time_value = row[Update_time_idx];
++fetched_rows;
const auto UPDATE_TIME_IDX = 12;
const auto & update_time_value = row[UPDATE_TIME_IDX];
if (!update_time_value.isNull())
{
update_time = update_time_value.getDateTime();
LOG_TRACE(log, "Got update time: " << update_time);
}
/// fetch remaining rows to avoid "commands out of sync" error
while (auto row = result.fetch());
while (auto row = result.fetch())
++fetched_rows;
}
if (0 == fetched_rows)
LOG_ERROR(log, "Cannot find table in SHOW TABLE STATUS result.");
if (fetched_rows > 1)
LOG_ERROR(log, "Found more than one table in SHOW TABLE STATUS result.");
}
catch (...)
{
@ -209,6 +239,7 @@ private:
const std::string db;
const std::string table;
const std::string where;
const bool dont_check_update_time;
Block sample_block;
mutable mysqlxx::PoolWithFailover pool;
const std::string load_all_query;

View File

@ -19,9 +19,10 @@ class RangeHashedDictionary final : public IDictionaryBase
public:
RangeHashedDictionary(
const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime)
const DictionaryLifetime dict_lifetime, bool require_nonempty)
: name{name}, dict_struct(dict_struct),
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime)
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty)
{
createAttributes();
@ -39,7 +40,7 @@ public:
}
RangeHashedDictionary(const RangeHashedDictionary & other)
: RangeHashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime}
: RangeHashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
{}
std::exception_ptr getCreationException() const override { return creation_exception; }
@ -218,6 +219,9 @@ private:
}
stream->readSuffix();
if (require_nonempty && 0 == element_count)
throw Exception("Dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY);
}
template <typename T>
@ -410,6 +414,7 @@ private:
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const bool require_nonempty;
std::map<std::string, std::size_t> attribute_index_by_name;
std::vector<attribute_t> attributes;
@ -417,7 +422,7 @@ private:
std::size_t bytes_allocated = 0;
std::size_t element_count = 0;
std::size_t bucket_count = 0;
mutable std::atomic<std::size_t> query_count{};
mutable std::atomic<std::size_t> query_count{0};
std::chrono::time_point<std::chrono::system_clock> creation_time;

View File

@ -43,6 +43,8 @@ namespace DB
* Например: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
* arrayEnumerateUniq(arr1, arr2...)
* - для кортежей из элементов на соответствующих позициях в нескольких массивах.
*
* emptyArrayToSingle(arr) - заменить пустые массивы на массивы из одного элемента со значением "по-умолчанию".
*/
@ -1695,13 +1697,263 @@ private:
};
class FunctionEmptyArrayToSingle : public IFunction
{
public:
static constexpr auto name = "emptyArrayToSingle";
static IFunction * create(const Context & context) { return new FunctionEmptyArrayToSingle; }
/// Получить имя функции.
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 DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
if (!array_type)
throw Exception("Argument for function " + getName() + " must be array.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return arguments[0]->clone();
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (executeConst(block, arguments, result))
return;
const ColumnArray * array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
if (!array)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
ColumnPtr res_ptr = array->cloneEmpty();
block.getByPosition(result).column = res_ptr;
ColumnArray & res = static_cast<ColumnArray &>(*res_ptr);
const IColumn & src_data = array->getData();
const ColumnArray::Offsets_t & src_offsets = array->getOffsets();
IColumn & res_data = res.getData();
ColumnArray::Offsets_t & res_offsets = res.getOffsets();
if (!( executeNumber<UInt8> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<UInt16> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<UInt32> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<UInt64> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<Int8> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<Int16> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<Int32> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<Int64> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<Float32> (src_data, src_offsets, res_data, res_offsets)
|| executeNumber<Float64> (src_data, src_offsets, res_data, res_offsets)
|| executeString (src_data, src_offsets, res_data, res_offsets)
|| executeFixedString (src_data, src_offsets, res_data, res_offsets)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
private:
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (const ColumnConstArray * const_array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get()))
{
if (const_array->getData().empty())
{
auto nested_type = typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType();
block.getByPosition(result).column = new ColumnConstArray(
block.rowsInFirstColumn(),
{nested_type->getDefault()},
nested_type->clone());
}
else
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
return true;
}
else
return false;
}
template <typename T>
bool executeNumber(
const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets,
IColumn & res_data_col, ColumnArray::Offsets_t & res_offsets)
{
if (const ColumnVector<T> * src_data_concrete = typeid_cast<const ColumnVector<T> *>(&src_data))
{
const PODArray<T> & src_data = src_data_concrete->getData();
PODArray<T> & res_data = typeid_cast<ColumnVector<T> &>(res_data_col).getData();
size_t size = src_offsets.size();
res_offsets.resize(size);
res_data.reserve(src_data.size());
ColumnArray::Offset_t src_prev_offset = 0;
ColumnArray::Offset_t res_prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
if (src_offsets[i] != src_prev_offset)
{
size_t size_to_write = src_offsets[i] - src_prev_offset;
size_t prev_res_data_size = res_data.size();
res_data.resize(prev_res_data_size + size_to_write);
memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset], size_to_write * sizeof(T));
res_prev_offset += size_to_write;
res_offsets[i] = res_prev_offset;
}
else
{
res_data.push_back(T());
++res_prev_offset;
res_offsets[i] = res_prev_offset;
}
src_prev_offset = src_offsets[i];
}
return true;
}
else
return false;
}
bool executeFixedString(
const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets,
IColumn & res_data_col, ColumnArray::Offsets_t & res_offsets)
{
if (const ColumnFixedString * src_data_concrete = typeid_cast<const ColumnFixedString *>(&src_data))
{
const size_t n = src_data_concrete->getN();
const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars();
ColumnFixedString::Chars_t & res_data = typeid_cast<ColumnFixedString &>(res_data_col).getChars();
size_t size = src_offsets.size();
res_offsets.resize(size);
res_data.reserve(src_data.size());
ColumnArray::Offset_t src_prev_offset = 0;
ColumnArray::Offset_t res_prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
if (src_offsets[i] != src_prev_offset)
{
size_t size_to_write = src_offsets[i] - src_prev_offset;
size_t prev_res_data_size = res_data.size();
res_data.resize(prev_res_data_size + size_to_write * n);
memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset], size_to_write * n);
res_prev_offset += size_to_write;
res_offsets[i] = res_prev_offset;
}
else
{
size_t prev_res_data_size = res_data.size();
res_data.resize(prev_res_data_size + n);
memset(&res_data[prev_res_data_size], 0, n);
++res_prev_offset;
res_offsets[i] = res_prev_offset;
}
src_prev_offset = src_offsets[i];
}
return true;
}
else
return false;
}
bool executeString(
const IColumn & src_data, const ColumnArray::Offsets_t & src_array_offsets,
IColumn & res_data_col, ColumnArray::Offsets_t & res_array_offsets)
{
if (const ColumnString * src_data_concrete = typeid_cast<const ColumnString *>(&src_data))
{
const ColumnString::Offsets_t & src_string_offsets = src_data_concrete->getOffsets();
ColumnString::Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res_data_col).getOffsets();
const ColumnString::Chars_t & src_data = src_data_concrete->getChars();
ColumnString::Chars_t & res_data = typeid_cast<ColumnString &>(res_data_col).getChars();
size_t size = src_array_offsets.size();
res_array_offsets.resize(size);
res_string_offsets.reserve(src_string_offsets.size());
res_data.reserve(src_data.size());
ColumnArray::Offset_t src_array_prev_offset = 0;
ColumnArray::Offset_t res_array_prev_offset = 0;
ColumnString::Offset_t src_string_prev_offset = 0;
ColumnString::Offset_t res_string_prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
if (src_array_offsets[i] != src_array_prev_offset)
{
size_t array_size = src_array_offsets[i] - src_array_prev_offset;
size_t bytes_to_copy = 0;
size_t from_string_prev_offset_local = src_string_prev_offset;
for (size_t j = 0; j < array_size; ++j)
{
size_t string_size = src_string_offsets[src_array_prev_offset + j] - from_string_prev_offset_local;
res_string_prev_offset += string_size;
res_string_offsets.push_back(res_string_prev_offset);
from_string_prev_offset_local += string_size;
bytes_to_copy += string_size;
}
size_t res_data_old_size = res_data.size();
res_data.resize(res_data_old_size + bytes_to_copy);
memcpy(&res_data[res_data_old_size], &src_data[src_string_prev_offset], bytes_to_copy);
res_array_prev_offset += array_size;
res_array_offsets[i] = res_array_prev_offset;
}
else
{
res_data.push_back(0); /// Пустая строка, включая ноль на конце.
++res_string_prev_offset;
res_string_offsets.push_back(res_string_prev_offset);
++res_array_prev_offset;
res_array_offsets[i] = res_array_prev_offset;
}
src_array_prev_offset = src_array_offsets[i];
if (src_array_prev_offset)
src_string_prev_offset = src_string_offsets[src_array_prev_offset - 1];
}
return true;
}
else
return false;
}
};
struct NameHas { static constexpr auto name = "has"; };
struct NameIndexOf { static constexpr auto name = "indexOf"; };
struct NameCountEqual { static constexpr auto name = "countEqual"; };
typedef FunctionArrayIndex<IndexToOne, NameHas> FunctionHas;
typedef FunctionArrayIndex<IndexToOne, NameHas> FunctionHas;
typedef FunctionArrayIndex<IndexIdentity, NameIndexOf> FunctionIndexOf;
typedef FunctionArrayIndex<IndexCount, NameCountEqual> FunctionCountEqual;
typedef FunctionArrayIndex<IndexCount, NameCountEqual> FunctionCountEqual;
using FunctionEmptyArrayUInt8 = FunctionEmptyArray<DataTypeUInt8>;
using FunctionEmptyArrayUInt16 = FunctionEmptyArray<DataTypeUInt16>;

View File

@ -862,7 +862,7 @@ private:
dict->getString(attr_name, ids, out.get());
block.getByPosition(result).column = new ColumnConst<String>{
id_col->size(), out->getDataAtWithTerminatingZero(0).toString()
id_col->size(), out->getDataAt(0).toString()
};
}
else
@ -967,7 +967,7 @@ private:
dictionary->getString(attr_name, ids, dates, out.get());
block.getByPosition(result).column = new ColumnConst<String>{
id_col->size(), out->getDataAtWithTerminatingZero(0).toString()
id_col->size(), out->getDataAt(0).toString()
};
}
else

View File

@ -404,31 +404,36 @@ struct ExtractURLParameterImpl
{
size_t cur_offset = offsets[i];
const char * str = reinterpret_cast<const char *>(&data[prev_offset]);
const char * pos = nullptr;
do
const char * begin = strchr(str, '?');
if (begin != nullptr)
{
const char * str = reinterpret_cast<const char *>(&data[prev_offset]);
const char * begin = strchr(str, '?');
if (begin == nullptr)
break;
pos = strstr(begin + 1, param_str);
if (pos == nullptr)
break;
if (pos != begin + 1 && *(pos - 1) != ';' && *(pos - 1) != '&')
pos = begin + 1;
while (true)
{
pos = nullptr;
break;
}
pos = strstr(pos, param_str);
pos += param_len;
} while (false);
if (pos == nullptr)
break;
if (pos[-1] != '?' && pos[-1] != '&')
{
pos += param_len;
continue;
}
else
{
pos += param_len;
break;
}
}
}
if (pos != nullptr)
{
const char * end = strpbrk(pos, "&;#");
const char * end = strpbrk(pos, "&#");
if (end == nullptr)
end = pos + strlen(pos);

View File

@ -3,7 +3,7 @@
#include <boost/noncopyable.hpp>
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/MemoryTracker.h>
#include <DB/Common/Allocator.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
@ -18,7 +18,7 @@ namespace DB
* Отличается тем, что не делает лишний memset. (И почти ничего не делает.)
* Также можно попросить выделять выровненный кусок памяти.
*/
struct Memory : boost::noncopyable
struct Memory : boost::noncopyable, Allocator
{
size_t m_capacity = 0;
size_t m_size = 0;
@ -66,16 +66,22 @@ struct Memory : boost::noncopyable
}
else
{
dealloc();
new_size = align(new_size);
m_data = reinterpret_cast<char *>(Allocator::realloc(m_data, m_capacity, new_size, alignment));
m_capacity = new_size;
m_size = m_capacity;
alloc();
}
}
private:
size_t align(size_t value) const
{
if (!alignment)
return value;
return (value + alignment - 1) / alignment * alignment;
}
void alloc()
{
if (!m_capacity)
@ -87,33 +93,10 @@ private:
ProfileEvents::increment(ProfileEvents::IOBufferAllocs);
ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, m_capacity);
if (current_memory_tracker)
current_memory_tracker->alloc(m_capacity);
char * new_m_data = nullptr;
if (!alignment)
{
new_m_data = reinterpret_cast<char *>(malloc(m_capacity));
if (!new_m_data)
throw Exception("Cannot allocate memory (malloc)", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
m_data = new_m_data;
return;
}
size_t aligned_capacity = (m_capacity + alignment - 1) / alignment * alignment;
m_capacity = aligned_capacity;
size_t new_capacity = align(m_capacity);
m_data = reinterpret_cast<char *>(Allocator::alloc(new_capacity, alignment));
m_capacity = new_capacity;
m_size = m_capacity;
int res = posix_memalign(reinterpret_cast<void **>(&new_m_data), alignment, m_capacity);
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign)", ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
m_data = new_m_data;
}
void dealloc()
@ -121,11 +104,8 @@ private:
if (!m_data)
return;
free(reinterpret_cast<void *>(m_data));
Allocator::free(reinterpret_cast<void *>(m_data), m_capacity);
m_data = nullptr; /// Чтобы избежать double free, если последующий вызов alloc кинет исключение.
if (current_memory_tracker)
current_memory_tracker->free(m_capacity);
}
};

View File

@ -15,8 +15,8 @@ template <class Buffer>
class IHashingBuffer : public BufferWithOwnMemory<Buffer>
{
public:
IHashingBuffer<Buffer>(size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE) :
block_pos(0), block_size(block_size_), state(0, 0)
IHashingBuffer<Buffer>(size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE)
: BufferWithOwnMemory<Buffer>(block_size_), block_pos(0), block_size(block_size_), state(0, 0)
{
}

View File

@ -674,17 +674,6 @@ typedef SharedPtr<AggregatedDataVariants> AggregatedDataVariantsPtr;
typedef std::vector<AggregatedDataVariantsPtr> ManyAggregatedDataVariants;
/** Достать вариант агрегации по его типу. */
template <typename Method> Method & getDataVariant(AggregatedDataVariants & variants);
#define M(NAME, IS_TWO_LEVEL) \
template <> inline decltype(AggregatedDataVariants::NAME)::element_type & getDataVariant<decltype(AggregatedDataVariants::NAME)::element_type>(AggregatedDataVariants & variants) { return *variants.NAME; }
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
/** Агрегирует источник блоков.
*/
class Aggregator
@ -733,11 +722,15 @@ public:
*/
AggregatedDataVariantsPtr merge(ManyAggregatedDataVariants & data_variants, size_t max_threads);
/** Объединить несколько агрегированных блоков в одну структуру данных.
/** Объединить поток частично агрегированных блоков в одну структуру данных.
* (Доагрегировать несколько блоков, которые представляют собой результат независимых агрегаций с удалённых серверов.)
*/
void mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads);
/** Объединить несколько частично агрегированных блоков в один.
*/
Block mergeBlocks(BlocksList & blocks, bool final);
using CancellationHook = std::function<bool()>;
/** Установить функцию, которая проверяет, можно ли прервать текущую задачу.
@ -974,4 +967,15 @@ protected:
};
/** Достать вариант агрегации по его типу. */
template <typename Method> Method & getDataVariant(AggregatedDataVariants & variants);
#define M(NAME, IS_TWO_LEVEL) \
template <> inline decltype(AggregatedDataVariants::NAME)::element_type & getDataVariant<decltype(AggregatedDataVariants::NAME)::element_type>(AggregatedDataVariants & variants) { return *variants.NAME; }
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}

View File

@ -66,6 +66,7 @@ public:
/// Для ARRAY_JOIN
NameSet array_joined_columns;
bool array_join_is_left;
/// Для JOIN
const Join * join = nullptr;
@ -122,13 +123,14 @@ public:
return a;
}
static ExpressionAction arrayJoin(const NameSet & array_joined_columns)
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left)
{
if (array_joined_columns.empty())
throw Exception("No arrays to join", ErrorCodes::LOGICAL_ERROR);
ExpressionAction a;
a.type = ARRAY_JOIN;
a.array_joined_columns = array_joined_columns;
a.array_join_is_left = array_join_is_left;
return a;
}

View File

@ -226,6 +226,10 @@ private:
/// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn.
void makeSet(ASTFunction * node, const Block & sample_block);
/// Замена скалярных подзапросов на значения-константы.
void executeScalarSubqueries();
void executeScalarSubqueriesImpl(ASTPtr & ast);
/// Находит глобальные подзапросы в секциях GLOBAL IN/JOIN. Заполняет external_tables.
void initGlobalSubqueriesAndExternalTables();
void initGlobalSubqueries(ASTPtr & ast);

View File

@ -57,9 +57,19 @@ private:
std::uint64_t error_count;
};
/** Имя словаря -> словарь.
*/
std::unordered_map<std::string, dictionary_info> dictionaries;
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
/** Здесь находятся словари, которых ещё ни разу не удалось загрузить.
* В dictionaries они тоже присутствуют, но с нулевым указателем dict.
*/
std::unordered_map<std::string, failed_dictionary_info> failed_dictionaries;
/** И для обычных и для failed_dictionaries.
*/
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
std::mt19937_64 rnd_engine{getSeed()};
Context & context;

View File

@ -24,8 +24,7 @@ public:
*/
BlockIO execute() override
{
executeImpl(false);
return {};
return executeImpl(false);
}
/** assume_metadata_exists - не проверять наличие файла с метаданными и не создавать его
@ -45,7 +44,7 @@ public:
const ColumnDefaults & column_defaults);
private:
void executeImpl(bool assume_metadata_exists);
BlockIO executeImpl(bool assume_metadata_exists);
/// AST в список столбцов с типами. Столбцы типа Nested развернуты в список настоящих столбцов.
using ColumnsAndDefaults = std::pair<NamesAndTypesList, ColumnDefaults>;

View File

@ -91,6 +91,8 @@ struct Settings
M(SettingUInt64, min_count_to_compile, 3) \
/** При каком количестве ключей, начинает использоваться двухуровневая агрегация. 0 - никогда не использовать. */ \
M(SettingUInt64, group_by_two_level_threshold, 100000) \
/** Включён ли экономный по памяти режим распределённой агрегации. */ \
M(SettingBool, distributed_aggregation_memory_efficient, false) \
\
/** Максимальное количество используемых реплик каждого шарда при выполнении запроса */ \
M(SettingUInt64, max_parallel_replicas, 1) \

View File

@ -18,4 +18,9 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit
*/
void stableSortBlock(Block & block, const SortDescription & description);
/** То же, что и stableSortBlock, но не сортировать блок, а только рассчитать перестановку значений,
* чтобы потом можно было переставить значения столбцов самостоятельно.
*/
void stableGetPermutation(const Block & block, const SortDescription & description, IColumn::Permutation & out_permutation);
}

View File

@ -1,6 +1,8 @@
#pragma once
#include <DB/Parsers/IAST.h>
#include <mysqlxx/Manip.h>
namespace DB
{
@ -98,5 +100,87 @@ public:
}
return res;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
frame.need_parens = false;
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : "");
if (!table.empty())
{
if (!database.empty())
{
settings.ostr << indent_str << database;
settings.ostr << ".";
}
settings.ostr << indent_str << table;
}
settings.ostr << settings.nl_or_ws;
for (size_t i = 0; i < parameters.size(); ++i)
{
const ASTAlterQuery::Parameters & p = parameters[i];
if (p.type == ASTAlterQuery::ADD_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (settings.hilite ? hilite_none : "");
p.col_decl->formatImpl(settings, state, frame);
/// AFTER
if (p.column)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : "");
p.column->formatImpl(settings, state, frame);
}
}
else if (p.type == ASTAlterQuery::DROP_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP COLUMN " << (settings.hilite ? hilite_none : "");
p.column->formatImpl(settings, state, frame);
}
else if (p.type == ASTAlterQuery::MODIFY_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (settings.hilite ? hilite_none : "");
p.col_decl->formatImpl(settings, state, frame);
}
else if (p.type == ASTAlterQuery::DROP_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (p.detach ? "DETACH" : "DROP") << " PARTITION "
<< (settings.hilite ? hilite_none : "");
p.partition->formatImpl(settings, state, frame);
}
else if (p.type == ASTAlterQuery::ATTACH_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ATTACH " << (p.unreplicated ? "UNREPLICATED " : "")
<< (p.part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : "");
p.partition->formatImpl(settings, state, frame);
}
else if (p.type == ASTAlterQuery::FETCH_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FETCH " << (p.unreplicated ? "UNREPLICATED " : "")
<< "PARTITION " << (settings.hilite ? hilite_none : "");
p.partition->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< " FROM " << (settings.hilite ? hilite_none : "") << mysqlxx::quote << p.from;
}
else if (p.type == ASTAlterQuery::FREEZE_PARTITION)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FREEZE PARTITION " << (settings.hilite ? hilite_none : "");
p.partition->formatImpl(settings, state, frame);
}
else
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
std::string comma = (i < (parameters.size() -1) ) ? "," : "";
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << comma << (settings.hilite ? hilite_none : "");
settings.ostr << settings.nl_or_ws;
}
}
};
}

View File

@ -16,6 +16,12 @@ public:
String getID() const override { return "Asterisk"; }
ASTPtr clone() const override { return new ASTAsterisk(*this); }
String getColumnName() const override { return "*"; }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << "*";
}
};
}

View File

@ -19,6 +19,28 @@ struct ASTCheckQuery : public IAST
std::string database;
std::string table;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
std::string nl_or_nothing = settings.one_line ? "" : "\n";
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
std::string nl_or_ws = settings.one_line ? " " : "\n";
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "CHECK TABLE " << (settings.hilite ? hilite_none : "");
if (!table.empty())
{
if (!database.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << database << (settings.hilite ? hilite_none : "");
settings.ostr << ".";
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << table << (settings.hilite ? hilite_none : "");
}
settings.ostr << nl_or_ws;
}
};
}

View File

@ -40,6 +40,26 @@ public:
return ptr;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
frame.need_parens = false;
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << settings.nl_or_ws << indent_str << backQuoteIfNeed(name);
if (type)
{
settings.ostr << ' ';
type->formatImpl(settings, state, frame);
}
if (default_expression)
{
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : "") << ' ';
default_expression->formatImpl(settings, state, frame);
}
}
};
}

View File

@ -30,7 +30,7 @@ public:
ASTCreateQuery() = default;
ASTCreateQuery(const StringRange range_) : IAST(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return (attach ? "AttachQuery_" : "CreateQuery_") + database + "_" + table; };
@ -48,6 +48,74 @@ public:
return ptr;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
frame.need_parens = false;
if (!database.empty() && table.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (attach ? "ATTACH DATABASE " : "CREATE DATABASE ") << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "")
<< backQuoteIfNeed(database);
return;
}
{
std::string what = "TABLE";
if (is_view)
what = "VIEW";
if (is_materialized_view)
what = "MATERIALIZED VIEW";
settings.ostr
<< (settings.hilite ? hilite_keyword : "")
<< (attach ? "ATTACH " : "CREATE ")
<< (is_temporary ? "TEMPORARY " : "")
<< what
<< " " << (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
}
if (!as_table.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "")
<< (!as_database.empty() ? backQuoteIfNeed(as_database) + "." : "") << backQuoteIfNeed(as_table);
}
if (columns)
{
settings.ostr << (settings.one_line ? " (" : "\n(");
FormatStateStacked frame_nested = frame;
++frame_nested.indent;
columns->formatImpl(settings, state, frame_nested);
settings.ostr << (settings.one_line ? ")" : "\n)");
}
if (storage && !is_materialized_view && !is_view)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ENGINE" << (settings.hilite ? hilite_none : "") << " = ";
storage->formatImpl(settings, state, frame);
}
if (inner_storage)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ENGINE" << (settings.hilite ? hilite_none : "") << " = ";
inner_storage->formatImpl(settings, state, frame);
}
if (is_populate)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " POPULATE" << (settings.hilite ? hilite_none : "");
}
if (select)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << settings.nl_or_ws << (settings.hilite ? hilite_none : "");
select->formatImpl(settings, state, frame);
}
}
};
}

View File

@ -24,6 +24,25 @@ public:
String getID() const override { return (detach ? "DetachQuery_" : "DropQuery_") + database + "_" + table; };
ASTPtr clone() const override { return new ASTDropQuery(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
if (table.empty() && !database.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< (detach ? "DETACH DATABASE " : "DROP DATABASE ")
<< (if_exists ? "IF EXISTS " : "")
<< (settings.hilite ? hilite_none : "")
<< backQuoteIfNeed(database);
return;
}
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< (detach ? "DETACH TABLE " : "DROP TABLE ")
<< (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
}
};
}

View File

@ -31,6 +31,40 @@ public:
return ptr;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
settings.ostr << ", ";
(*it)->formatImpl(settings, state, frame);
}
}
friend class ASTSelectQuery;
/** Вывести список выражений в секциях запроса SELECT - по одному выражению на строку.
*/
void formatImplMultiline(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
std::string indent_str = "\n" + std::string(4 * (frame.indent + 1), ' ');
++frame.indent;
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
settings.ostr << ", ";
if (children.size() > 1)
settings.ostr << indent_str;
(*it)->formatImpl(settings, state, frame);
}
}
};
}

View File

@ -82,6 +82,9 @@ public:
return ptr;
}
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};

View File

@ -2,6 +2,7 @@
#include <DB/DataTypes/IDataType.h>
#include <DB/Parsers/ASTWithAlias.h>
#include <DB/IO/WriteBufferFromOStream.h>
namespace DB
@ -41,6 +42,18 @@ public:
{
set.insert(name);
}
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
WriteBufferFromOStream wb(settings.ostr, 32);
writeProbablyBackQuotedString(name, wb);
wb.next();
settings.ostr << (settings.hilite ? hilite_none : "");
}
};
}

View File

@ -42,6 +42,43 @@ public:
return ptr;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
frame.need_parens = false;
settings.ostr << (settings.hilite ? hilite_keyword : "") << "INSERT INTO " << (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
if (!insert_id.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ID = " << (settings.hilite ? hilite_none : "")
<< mysqlxx::quote << insert_id;
if (columns)
{
settings.ostr << " (";
columns->formatImpl(settings, state, frame);
settings.ostr << ")";
}
if (select)
{
settings.ostr << " ";
select->formatImpl(settings, state, frame);
}
else
{
if (!format.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FORMAT " << (settings.hilite ? hilite_none : "") << format;
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " VALUES" << (settings.hilite ? hilite_none : "");
}
}
}
};
}

View File

@ -83,6 +83,37 @@ public:
return ptr;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
frame.need_parens = false;
settings.ostr << (settings.hilite ? hilite_keyword : "");
if (locality == ASTJoin::Global)
settings.ostr << "GLOBAL ";
if (kind != ASTJoin::Cross)
settings.ostr << (strictness == ASTJoin::Any ? "ANY " : "ALL ");
settings.ostr << (kind == ASTJoin::Inner ? "INNER "
: (kind == ASTJoin::Left ? "LEFT "
: (kind == ASTJoin::Right ? "RIGHT "
: (kind == ASTJoin::Cross ? "CROSS "
: "FULL OUTER "))));
settings.ostr << "JOIN "
<< (settings.hilite ? hilite_none : "");
table->formatImpl(settings, state, frame);
if (kind != ASTJoin::Cross)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " USING " << (settings.hilite ? hilite_none : "");
using_expr_list->formatImpl(settings, state, frame);
}
}
};
}

View File

@ -26,6 +26,12 @@ public:
String getID() const override { return "Literal_" + apply_visitor(FieldVisitorDump(), value); }
ASTPtr clone() const override { return new ASTLiteral(*this); }
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << apply_visitor(FieldVisitorToString(), value);
}
};
}

View File

@ -34,6 +34,15 @@ public:
return ptr;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << settings.nl_or_ws << indent_str << backQuoteIfNeed(name) << " ";
type->formatImpl(settings, state, frame);
}
};
}

View File

@ -22,6 +22,13 @@ public:
String getID() const override { return "OptimizeQuery_" + database + "_" + table; };
ASTPtr clone() const override { return new ASTOptimizeQuery(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
}
};
}

View File

@ -29,6 +29,18 @@ public:
String getID() const override { return "OrderByElement"; }
ASTPtr clone() const override { return new ASTOrderByElement(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
children.front()->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << (direction == -1 ? " DESC" : " ASC") << (settings.hilite ? hilite_none : "");
if (!collator.isNull())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "")
<< "'" << collator->getLocale() << "'";
}
}
};
}

View File

@ -24,7 +24,7 @@ public:
/// Объявляет класс-наследник ASTQueryWithOutput с реализованными методами getID и clone.
#define DEFINE_AST_QUERY_WITH_OUTPUT(Name, ID) \
#define DEFINE_AST_QUERY_WITH_OUTPUT(Name, ID, Query) \
class Name : public ASTQueryWithOutput \
{ \
public: \
@ -44,6 +44,12 @@ public: \
} \
return ptr; \
} \
\
protected: \
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override \
{ \
settings.ostr << (settings.hilite ? hilite_keyword : "") << Query << (settings.hilite ? hilite_none : ""); \
} \
};
}

View File

@ -6,26 +6,40 @@
namespace DB
{
/** Запрос с указанием названия таблицы и, возможно, БД и секцией FORMAT.
*/
class ASTQueryWithTableAndOutput : public ASTQueryWithOutput
/** Запрос с указанием названия таблицы и, возможно, БД и секцией FORMAT.
*/
class ASTQueryWithTableAndOutput : public ASTQueryWithOutput
{
public:
String database;
String table;
ASTQueryWithTableAndOutput() = default;
ASTQueryWithTableAndOutput(const StringRange range_) : ASTQueryWithOutput(range_) {}
protected:
void formatHelper(const FormatSettings & settings, FormatState & state, FormatStateStacked frame, const char * name) const
{
public:
String database;
String table;
ASTQueryWithTableAndOutput() = default;
ASTQueryWithTableAndOutput(const StringRange range_) : ASTQueryWithOutput(range_) {}
};
settings.ostr << (settings.hilite ? hilite_keyword : "") << name << " " << (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
if (format)
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << indent_str << "FORMAT " << (settings.hilite ? hilite_none : "");
format->formatImpl(settings, state, frame);
}
}
};
/// Объявляет класс-наследник ASTQueryWithTableAndOutput с реализованными методами getID и clone.
#define DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(Name, ID) \
#define DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(Name, ID, Query) \
class Name : public ASTQueryWithTableAndOutput \
{ \
public: \
public: \
Name() = default; \
Name(const StringRange range_) : ASTQueryWithTableAndOutput(range_) {} \
String getID() const override { return ID"_" + database + "_" + table; }; \
@ -42,5 +56,11 @@ public: \
} \
return ptr; \
} \
\
protected: \
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override \
{ \
formatHelper(settings, state, frame, Query); \
} \
};
}

View File

@ -34,6 +34,22 @@ public:
String getID() const override { return "Rename"; };
ASTPtr clone() const override { return new ASTRenameQuery(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME TABLE " << (settings.hilite ? hilite_none : "");
for (ASTRenameQuery::Elements::const_iterator it = elements.begin(); it != elements.end(); ++it)
{
if (it != elements.begin())
settings.ostr << ", ";
settings.ostr << (!it->from.database.empty() ? backQuoteIfNeed(it->from.database) + "." : "") << backQuoteIfNeed(it->from.table)
<< (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "")
<< (!it->to.database.empty() ? backQuoteIfNeed(it->to.database) + "." : "") << backQuoteIfNeed(it->to.table);
}
}
};
}

View File

@ -50,6 +50,7 @@ public:
ASTPtr select_expression_list;
ASTPtr database;
ASTPtr table; /// Идентификатор, табличная функция или подзапрос (рекурсивно ASTSelectQuery)
bool array_join_is_left = false; /// LEFT ARRAY JOIN
ASTPtr array_join_expression_list; /// ARRAY JOIN
ASTPtr join; /// Обычный (не ARRAY) JOIN.
bool final = false;
@ -67,6 +68,9 @@ public:
ASTPtr prev_union_all;
/// Следующий запрос SELECT в цепочке UNION ALL, если такой есть
ASTPtr next_union_all;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -22,6 +22,17 @@ public:
String getID() const override { return "Set_" + getColumnName(); }
ASTPtr clone() const override { return new ASTSet(*this); }
String getColumnName() const override { return column_name; }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
/** Подготовленное множество. В пользовательских запросах такого не бывает, но такое бывает после промежуточных преобразований запроса.
* Выведем его не по-настоящему (это не будет корректным запросом, но покажет, что здесь было множество).
*/
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< "(...)"
<< (settings.hilite ? hilite_none : "");
}
};
}

View File

@ -26,11 +26,25 @@ public:
ASTSetQuery() = default;
ASTSetQuery(const StringRange range_) : IAST(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "Set"; };
ASTPtr clone() const override { return new ASTSetQuery(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SET " << (global ? "GLOBAL " : "") << (settings.hilite ? hilite_none : "");
for (ASTSetQuery::Changes::const_iterator it = changes.begin(); it != changes.end(); ++it)
{
if (it != changes.begin())
settings.ostr << ", ";
settings.ostr << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value);
}
}
};
}

View File

@ -5,5 +5,7 @@
namespace DB
{
DEFINE_AST_QUERY_WITH_OUTPUT(ASTShowProcesslistQuery, "ShowProcesslistQuery")
DEFINE_AST_QUERY_WITH_OUTPUT(ASTShowProcesslistQuery, "ShowProcesslistQuery", "SHOW PROCESSLIST")
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <mysqlxx/Manip.h>
#include <DB/Parsers/IAST.h>
#include <DB/Parsers/ASTQueryWithOutput.h>
@ -20,7 +21,7 @@ public:
ASTShowTablesQuery() = default;
ASTShowTablesQuery(const StringRange range_) : ASTQueryWithOutput(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "ShowTables"; };
@ -30,15 +31,43 @@ public:
ASTPtr ptr{res};
res->children.clear();
if (format)
{
res->format = format->clone();
res->children.push_back(res->format);
}
return ptr;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
if (databases)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW DATABASES" << (settings.hilite ? hilite_none : "");
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW TABLES" << (settings.hilite ? hilite_none : "");
if (!from.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "")
<< backQuoteIfNeed(from);
if (!like.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIKE " << (settings.hilite ? hilite_none : "")
<< mysqlxx::quote << like;
}
if (format)
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << indent_str << "FORMAT " << (settings.hilite ? hilite_none : "");
format->formatImpl(settings, state, frame);
}
}
};
}

View File

@ -2,7 +2,7 @@
#include <DB/DataTypes/IDataType.h>
#include <DB/Parsers/IAST.h>
#include <DB/Parsers/ASTWithAlias.h>
namespace DB
@ -11,12 +11,12 @@ namespace DB
/** Подзарос SELECT
*/
class ASTSubquery : public IAST
class ASTSubquery : public ASTWithAlias
{
public:
ASTSubquery() = default;
ASTSubquery(const StringRange range_) : IAST(range_) {}
ASTSubquery(const StringRange range_) : ASTWithAlias(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "Subquery"; }
@ -34,6 +34,20 @@ public:
}
String getColumnName() const override { return getTreeID(); }
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
std::string nl_or_nothing = settings.one_line ? "" : "\n";
settings.ostr << nl_or_nothing << indent_str << "(" << nl_or_nothing;
FormatStateStacked frame_nested = frame;
frame_nested.need_parens = false;
++frame_nested.indent;
children[0]->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent_str << ")";
}
};
}

View File

@ -16,11 +16,18 @@ public:
ASTUseQuery() = default;
ASTUseQuery(const StringRange range_) : IAST(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "UseQuery_" + database; };
ASTPtr clone() const override { return new ASTUseQuery(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database);
return;
}
};
}

View File

@ -1,11 +1,13 @@
#pragma once
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Parsers/IAST.h>
namespace DB
{
/** Базовый класс для AST, которые могут содержать алиас (идентификаторы, литералы, функции).
*/
class ASTWithAlias : public IAST
@ -19,10 +21,16 @@ public:
String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; }
String tryGetAlias() const override { return alias; }
void setAlias(const String & to) override { alias = to; }
/// Вызывает formatImplWithoutAlias, а также выводит алиас. Если надо - заключает всё выражение в скобки.
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override final;
virtual void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const = 0;
};
/// helper for setting aliases and chaining result to other functions
inline ASTPtr setAlias(ASTPtr ast, const String & alias) {
inline ASTPtr setAlias(ASTPtr ast, const String & alias)
{
ast->setAlias(alias);
return ast;
};

View File

@ -3,6 +3,8 @@
#include <list>
#include <set>
#include <sstream>
#include <iostream>
#include <set>
#include <Poco/SharedPtr.h>
@ -14,8 +16,6 @@
#include <DB/IO/WriteHelpers.h>
#include <DB/Parsers/StringRange.h>
#include <iostream>
namespace DB
{
@ -133,6 +133,68 @@ public:
(*it)->collectIdentifierNames(set);
}
/// Преобразовать в строку.
/// Настройки формата.
struct FormatSettings
{
std::ostream & ostr;
bool hilite;
bool one_line;
char nl_or_ws;
FormatSettings(std::ostream & ostr_, bool hilite_, bool one_line_)
: ostr(ostr_), hilite(hilite_), one_line(one_line_)
{
nl_or_ws = one_line ? ' ' : '\n';
}
};
/// Состояние. Например, может запоминаться множество узлов, которых мы уже обошли.
struct FormatState
{
/** Запрос SELECT, в котором найден алиас; идентификатор узла с таким алиасом.
* Нужно, чтобы когда узел встретился повторно, выводить только алиас.
*/
std::set<std::pair<const IAST *, std::string>> printed_asts_with_alias;
};
/// Состояние, которое копируется при форматировании каждого узла. Например, уровень вложенности.
struct FormatStateStacked
{
UInt8 indent = 0;
bool need_parens = false;
const IAST * current_select = nullptr;
};
void format(const FormatSettings & settings) const
{
FormatState state;
formatImpl(settings, state, FormatStateStacked());
}
virtual void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
throw Exception("Unknown element in AST: " + getID()
+ ((range.first && (range.second > range.first))
? " '" + std::string(range.first, range.second - range.first) + "'"
: ""),
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
}
void writeAlias(const String & name, std::ostream & s, bool hilite) const;
protected:
/// Для подсветки синтаксиса.
static const char * hilite_keyword;
static const char * hilite_identifier;
static const char * hilite_function;
static const char * hilite_operator;
static const char * hilite_alias;
static const char * hilite_none;
private:
size_t checkDepthImpl(size_t max_depth, size_t level) const
{
@ -152,4 +214,9 @@ private:
typedef SharedPtr<IAST> ASTPtr;
typedef std::vector<ASTPtr> ASTs;
/// Квотировать идентификатор обратными кавычками, если это требуется.
String backQuoteIfNeed(const String & x);
}

View File

@ -5,17 +5,9 @@
namespace DB
{
/** EXISTS запрос
*/
DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTExistsQuery, "ExistsQuery")
/** SHOW CREATE TABLE запрос
*/
DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTShowCreateQuery, "ShowCreateQuery")
/** DESCRIBE TABLE запрос
*/
DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTDescribeQuery, "DescribeQuery")
DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTExistsQuery, "ExistsQuery", "EXISTS TABLE")
DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTShowCreateQuery, "ShowCreateQuery", "SHOW CREATE TABLE")
DEFINE_AST_QUERY_WITH_TABLE_AND_OUTPUT(ASTDescribeQuery, "DescribeQuery", "DESCRIBE TABLE")
}

View File

@ -12,11 +12,14 @@ namespace DB
/** Берёт синтаксическое дерево и превращает его обратно в текст.
* В случае запроса INSERT, данные будут отсутствовать.
*/
void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
inline void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false)
{
IAST::FormatSettings settings(s, hilite, one_line);
ast.format(settings);
}
String formatColumnsForCreateQuery(NamesAndTypesList & columns);
String backQuoteIfNeed(const String & x);
inline std::ostream & operator<<(std::ostream & os, const IAST & ast) { return formatAST(ast, os, 0, false, true), os; }
inline std::ostream & operator<<(std::ostream & os, const ASTPtr & ast) { return formatAST(*ast, os, 0, false, true), os; }

View File

@ -6,34 +6,13 @@
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/SipHash.h>
#include <DB/Interpreters/AggregationCommon.h>
#include <DB/DataStreams/MarkInCompressedFile.h>
namespace DB
{
struct MarkInCompressedFile
{
size_t offset_in_compressed_file;
size_t offset_in_decompressed_block;
bool operator==(const MarkInCompressedFile & rhs) const
{
return std::forward_as_tuple(offset_in_compressed_file, offset_in_decompressed_block) ==
std::forward_as_tuple(rhs.offset_in_compressed_file, rhs.offset_in_decompressed_block);
}
bool operator!=(const MarkInCompressedFile & rhs) const
{
return !(*this == rhs);
}
String toString() const
{
return "(" + DB::toString(offset_in_compressed_file) + "," + DB::toString(offset_in_decompressed_block) + ")";
}
};
typedef std::vector<MarkInCompressedFile> MarksInCompressedFile;
/// Оценка количества байтов, занимаемых засечками в кеше.
struct MarksWeightFunction
{

View File

@ -23,19 +23,16 @@ public:
{
DayNum_t left_date;
DayNum_t right_date;
UInt64 left;
UInt64 right;
Int64 left;
Int64 right;
UInt32 level;
std::string name;
DayNum_t left_month;
DayNum_t right_month;
DayNum_t month;
bool operator<(const Part & rhs) const
{
if (left_month != rhs.left_month)
return left_month < rhs.left_month;
if (right_month != rhs.right_month)
return right_month < rhs.right_month;
if (month != rhs.month)
return month < rhs.month;
if (left != rhs.left)
return left < rhs.left;
@ -48,8 +45,7 @@ public:
/// Содержит другой кусок (получен после объединения другого куска с каким-то ещё)
bool contains(const Part & rhs) const
{
return left_month == rhs.left_month /// Куски за разные месяцы не объединяются
&& right_month == rhs.right_month
return month == rhs.month /// Куски за разные месяцы не объединяются
&& left_date <= rhs.left_date
&& right_date >= rhs.right_date
&& left <= rhs.left
@ -66,7 +62,7 @@ public:
size_t size() const;
static String getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level);
static String getPartName(DayNum_t left_date, DayNum_t right_date, Int64 left_id, Int64 right_id, UInt64 level);
/// Возвращает true если имя директории совпадает с форматом имени директории кусочков
static bool isPartDirectory(const String & dir_name, Poco::RegularExpression::MatchVec * out_matches = nullptr);

View File

@ -33,7 +33,7 @@ public:
auto part_blocks = storage.writer.splitBlockIntoParts(block);
for (auto & current_block : part_blocks)
{
UInt64 temp_index = storage.increment.get();
Int64 temp_index = storage.increment.get();
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, temp_index);
storage.data.renameTempPartAndAdd(part, &storage.increment);

View File

@ -625,7 +625,7 @@ public:
|| mode == Mode::Aggregating;
}
UInt64 getMaxDataPartIndex();
Int64 getMaxDataPartIndex();
std::string getTableName() const override
{
@ -779,6 +779,13 @@ public:
return it == std::end(column_sizes) ? 0 : it->second;
}
using ColumnSizes = std::unordered_map<std::string, size_t>;
ColumnSizes getColumnSizes() const
{
Poco::ScopedLock<Poco::FastMutex> lock{data_parts_mutex};
return column_sizes;
}
/// Для ATTACH/DETACH/DROP PARTITION.
static String getMonthName(const Field & partition);
static DayNum_t getMonthDayNum(const Field & partition);
@ -810,7 +817,7 @@ private:
NamesAndTypesListPtr columns;
/// Актуальные размеры столбцов в сжатом виде
std::unordered_map<std::string, size_t> column_sizes;
ColumnSizes column_sizes;
BrokenPartCallback broken_part_callback;

View File

@ -43,7 +43,7 @@ public:
* temp_index - значение left и right для нового куска. Можно будет изменить при переименовании.
* Возвращает кусок с именем, начинающимся с tmp_, еще не добавленный в MergeTreeData.
*/
MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithDateInterval & block, UInt64 temp_index);
MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithDateInterval & block, Int64 temp_index);
private:
MergeTreeData & data;

View File

@ -247,7 +247,7 @@ private:
++right;
}
/// Если правее засечек нет, просто используем DEFAULT_BUFFER_SIZE
/// Если правее засечек нет, просто используем max_read_buffer_size
if (right >= (*marks).size() || (right + 1 == (*marks).size() &&
(*marks)[right].offset_in_compressed_file == (*marks)[all_mark_ranges[i].end].offset_in_compressed_file))
{

View File

@ -14,6 +14,8 @@
namespace DB
{
class IMergedBlockOutputStream : public IBlockOutputStream
{
public:
@ -230,7 +232,9 @@ protected:
CompressionMethod compression_method;
};
/** Для записи одного куска. Данные уже отсортированы, относятся к одному месяцу, и пишутся в один кускок.
/** Для записи одного куска.
* Данные относятся к одному месяцу, и пишутся в один кускок.
*/
class MergedBlockOutputStream : public IMergedBlockOutputStream
{
@ -278,45 +282,18 @@ public:
}
}
/// Если данные заранее отсортированы.
void write(const Block & block) override
{
size_t rows = block.rows();
writeImpl(block, nullptr);
}
/// Сначала пишем индекс. Индекс содержит значение Primary Key для каждой index_granularity строки.
typedef std::vector<const ColumnWithTypeAndName *> PrimaryColumns;
PrimaryColumns primary_columns;
for (const auto & descr : storage.getSortDescription())
primary_columns.push_back(
!descr.column_name.empty()
? &block.getByName(descr.column_name)
: &block.getByPosition(descr.column_number));
for (size_t i = index_offset; i < rows; i += storage.index_granularity)
{
for (PrimaryColumns::const_iterator it = primary_columns.begin(); it != primary_columns.end(); ++it)
{
if (storage.mode != MergeTreeData::Unsorted)
index_vec.push_back((*(*it)->column)[i]);
(*it)->type->serializeBinary(index_vec.back(), *index_stream);
}
++marks_count;
}
/// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз
OffsetColumns offset_columns;
/// Теперь пишем данные.
for (const auto & it : columns_list)
{
const ColumnWithTypeAndName & column = block.getByName(it.name);
writeData(column.name, *column.type, *column.column, offset_columns);
}
size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity;
index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity;
/** Если данные не отсортированы, но мы заранее вычислили перестановку, после которой они станут сортированными.
* Этот метод используется для экономии оперативки, так как не нужно держать одновременно два блока - исходный и отсортированный.
*/
void writeWithPermutation(const Block & block, const IColumn::Permutation * permutation)
{
writeImpl(block, permutation);
}
void writeSuffix() override
@ -391,6 +368,86 @@ private:
}
}
/** Если задана permutation, то переставляет значения в столбцах при записи.
* Это нужно, чтобы не держать целый блок в оперативке для его сортировки.
*/
void writeImpl(const Block & block, const IColumn::Permutation * permutation)
{
size_t rows = block.rows();
/// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз
OffsetColumns offset_columns;
auto sort_description = storage.getSortDescription();
/// Сюда будем складывать столбцы, относящиеся к Primary Key, чтобы потом записать индекс.
std::vector<ColumnWithTypeAndName> primary_columns(sort_description.size());
std::map<String, size_t> primary_columns_name_to_position;
for (size_t i = 0, size = sort_description.size(); i < size; ++i)
{
const auto & descr = sort_description[i];
String name = !descr.column_name.empty()
? descr.column_name
: block.getByPosition(descr.column_number).name;
if (!primary_columns_name_to_position.emplace(name, i).second)
throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
primary_columns[i] = !descr.column_name.empty()
? block.getByName(descr.column_name)
: block.getByPosition(descr.column_number);
/// Столбцы первичного ключа переупорядочиваем заранее и складываем в primary_columns.
if (permutation)
primary_columns[i].column = primary_columns[i].column->permute(*permutation, 0);
}
/// Теперь пишем данные.
for (const auto & it : columns_list)
{
const ColumnWithTypeAndName & column = block.getByName(it.name);
if (permutation)
{
auto primary_column_it = primary_columns_name_to_position.find(it.name);
if (primary_columns_name_to_position.end() != primary_column_it)
{
writeData(column.name, *column.type, *primary_columns[primary_column_it->second].column, offset_columns);
}
else
{
/// Столбцы, не входящие в первичный ключ, переупорядочиваем здесь; затем результат освобождается - для экономии оперативки.
ColumnPtr permutted_column = column.column->permute(*permutation, 0);
writeData(column.name, *column.type, *permutted_column, offset_columns);
}
}
else
{
writeData(column.name, *column.type, *column.column, offset_columns);
}
}
/// Пишем индекс. Индекс содержит значение Primary Key для каждой index_granularity строки.
for (size_t i = index_offset; i < rows; i += storage.index_granularity)
{
if (storage.mode != MergeTreeData::Unsorted)
{
for (const auto & primary_column : primary_columns)
{
index_vec.push_back((*primary_column.column)[i]);
primary_column.type->serializeBinary(index_vec.back(), *index_stream);
}
}
++marks_count;
}
size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity;
index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity;
}
private:
NamesAndTypesList columns_list;
String part_path;

View File

@ -38,7 +38,7 @@ public:
AbandonableLockInZooKeeper block_number_lock = storage.allocateBlockNumber(month_name);
UInt64 part_number = block_number_lock.getNumber();
Int64 part_number = block_number_lock.getNumber();
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, part_number);
String part_name = ActiveDataPartSet::getPartName(part->left_date, part->right_date, part->left, part->right, part->level);

View File

@ -84,8 +84,6 @@ public:
};
typedef std::map<String, ColumnData> Files_t;
Files_t & getFiles() { return files; }
bool checkData() const override;
protected:
@ -149,7 +147,7 @@ private:
size_t max_compress_block_size;
protected:
FileChecker<StorageLog> file_checker;
FileChecker file_checker;
private:
/** Для обычных столбцов, в засечках указано количество строчек в блоке.

View File

@ -393,8 +393,11 @@ private:
*/
void waitForReplicaToProcessLogEntry(const String & replica_name, const LogEntry & entry);
/// Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper.
static String padIndex(UInt64 index)
/** Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper.
* Поддерживаются также отрицательные числа - для них имя ноды выглядит несколько глупо
* и не соответствует никакой автоинкрементной ноде в ZK.
*/
static String padIndex(Int64 index)
{
String index_str = toString(index);
return std::string(10 - index_str.size(), '0') + index_str;

View File

@ -0,0 +1,89 @@
#pragma once
#include <map>
#include <Poco/File.h>
#include <DB/Storages/IStorage.h>
#include <DB/Common/FileChecker.h>
namespace DB
{
/** Реализует хранилище, подходящее для маленьких кусочков лога.
* При этом, хранит все столбцы в одном файле формата Native, с расположенным рядом индексом.
*/
class StorageStripeLog : public IStorage
{
friend class StripeLogBlockInputStream;
friend class StripeLogBlockOutputStream;
public:
/** Подцепить таблицу с соответствующим именем, по соответствующему пути (с / на конце),
* (корректность имён и путей не проверяется)
* состоящую из указанных столбцов.
* Если не указано attach - создать директорию, если её нет.
*/
static StoragePtr create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_ = DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
std::string getName() const override { return "StripeLog"; }
std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
BlockInputStreams read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query) override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
bool checkData() const override;
/// Данные файла.
struct ColumnData
{
Poco::File data_file;
};
typedef std::map<String, ColumnData> Files_t;
std::string full_path() { return path + escapeForFileName(name) + '/';}
private:
String path;
String name;
NamesAndTypesListPtr columns;
size_t max_compress_block_size;
FileChecker file_checker;
Poco::RWLock rwlock;
Logger * log;
StorageStripeLog(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_);
};
}

View File

@ -64,8 +64,6 @@ public:
};
typedef std::map<String, ColumnData> Files_t;
Files_t & getFiles();
std::string full_path() { return path + escapeForFileName(name) + '/';}
private:
@ -77,7 +75,7 @@ private:
Files_t files;
FileChecker<StorageTinyLog> file_checker;
FileChecker file_checker;
Logger * log;

View File

@ -652,7 +652,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("State")), argument_types, recursion_level + 1);
return new AggregateFunctionState(nested);
}
else if (recursion_level == 0 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge")))
else if (recursion_level <= 1 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge")))
{
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
if (argument_types.size() != 1)
@ -668,7 +668,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
return new AggregateFunctionMerge(nested);
}
else if (recursion_level <= 1 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
else if (recursion_level <= 2 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
{
if (argument_types.empty())
throw Exception{
@ -682,7 +682,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
AggregateFunctionPtr nested = get(String(name.data(), name.size() - 2), nested_dt, recursion_level + 1);
return new AggregateFunctionIf(nested);
}
else if (recursion_level <= 2 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array")))
else if (recursion_level <= 3 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array")))
{
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
size_t num_agruments = argument_types.size();
@ -695,7 +695,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
else
throw Exception("Illegal type " + argument_types[i]->getName() + " of argument #" + toString(i + 1) + " for aggregate function " + name + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("Array")), nested_arguments, recursion_level + 2); /// + 2, чтобы ни один другой модификатор не мог идти перед Array
AggregateFunctionPtr nested = get(String(name.data(), name.size() - strlen("Array")), nested_arguments, recursion_level + 3); /// + 3, чтобы ни один другой модификатор не мог идти перед Array
return new AggregateFunctionArray(nested);
}
else
@ -765,14 +765,14 @@ bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int
if (recursion_level <= 0 && name.size() > strlen("State") && !(strcmp(name.data() + name.size() - strlen("State"), "State")))
return isAggregateFunctionName(String(name.data(), name.size() - strlen("State")), recursion_level + 1);
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
if (recursion_level <= 0 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge")))
if (recursion_level <= 1 && name.size() > strlen("Merge") && !(strcmp(name.data() + name.size() - strlen("Merge"), "Merge")))
return isAggregateFunctionName(String(name.data(), name.size() - strlen("Merge")), recursion_level + 1);
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
if (recursion_level <= 1 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
if (recursion_level <= 2 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
return isAggregateFunctionName(String(name.data(), name.size() - 2), recursion_level + 1);
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
if (recursion_level <= 2 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array")))
return isAggregateFunctionName(String(name.data(), name.size() - strlen("Array")), recursion_level + 2); /// + 2, чтобы ни один другой модификатор не мог идти перед Array
if (recursion_level <= 3 && name.size() > strlen("Array") && !(strcmp(name.data() + name.size() - strlen("Array"), "Array")))
return isAggregateFunctionName(String(name.data(), name.size() - strlen("Array")), recursion_level + 3); /// + 3, чтобы ни один другой модификатор не мог идти перед Array
return false;
}

View File

@ -369,6 +369,19 @@ private:
}
/** Проверка для случая, когда в терминал вставляется многострочный запрос из буфера обмена.
* Позволяет не начинать выполнение одной строчки запроса, пока весь запрос не будет вставлен.
*/
static bool hasDataInSTDIN()
{
timeval timeout = { 0, 0 };
fd_set fds;
FD_ZERO(&fds);
FD_SET(STDIN_FILENO, &fds);
return select(1, &fds, 0, 0, &timeout) == 1;
}
void loop()
{
String query;
@ -395,7 +408,7 @@ private:
query += line;
if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || !config().has("multiline")))
if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || (!config().has("multiline") && !hasDataInSTDIN())))
{
if (query != prev_query)
{
@ -464,6 +477,12 @@ private:
copyData(in, out);
}
process(line);
}
bool process(const String & line)
{
if (config().has("multiquery"))
{
/// Несколько запросов, разделенных ';'.
@ -494,17 +513,20 @@ private:
while (isWhitespace(*begin) || *begin == ';')
++begin;
process(query, ast);
if (!processSingleQuery(query, ast))
return false;
}
return true;
}
else
{
process(line);
return processSingleQuery(line);
}
}
bool process(const String & line, ASTPtr parsed_query_ = nullptr)
bool processSingleQuery(const String & line, ASTPtr parsed_query_ = nullptr)
{
if (exit_strings.end() != exit_strings.find(line))
return false;
@ -838,15 +860,8 @@ private:
}
void onData(Block & block)
void initBlockOutputStream(const Block & block)
{
if (written_progress_chars)
clearProgress();
if (!block)
return;
processed_rows += block.rows();
if (!block_std_out)
{
String current_format = format;
@ -869,8 +884,21 @@ private:
block_std_out = context.getFormatFactory().getOutput(current_format, std_out, block);
block_std_out->writePrefix();
}
}
/// Загаловочный блок с нулем строк использовался для инициализации block_std_out,
void onData(Block & block)
{
if (written_progress_chars)
clearProgress();
if (!block)
return;
processed_rows += block.rows();
initBlockOutputStream(block);
/// Заголовочный блок с нулем строк использовался для инициализации block_std_out,
/// выводить его не нужно
if (block.rows() != 0)
{
@ -885,11 +913,13 @@ private:
void onTotals(Block & block)
{
initBlockOutputStream(block);
block_std_out->setTotals(block);
}
void onExtremes(Block & block)
{
initBlockOutputStream(block);
block_std_out->setExtremes(block);
}

View File

@ -214,7 +214,7 @@ Connection::Packet ParallelReplicas::receivePacketUnlocked()
auto it = getReplicaForReading();
if (it == replica_map.end())
throw Exception("No available replica", ErrorCodes::NO_AVAILABLE_REPLICA);
throw Exception("Logical error: no available replica", ErrorCodes::NO_AVAILABLE_REPLICA);
Connection * connection = it->second;
Connection::Packet packet = connection->receivePacket();
@ -263,9 +263,8 @@ ParallelReplicas::ReplicaMap::iterator ParallelReplicas::waitForReadEvent()
Poco::Net::Socket::SocketList read_list;
read_list.reserve(active_replica_count);
/** Сначала проверяем, есть ли данные, которые уже лежат в буфере
* хоть одного соединения.
*/
/// Сначала проверяем, есть ли данные, которые уже лежат в буфере
/// хоть одного соединения.
for (auto & e : replica_map)
{
Connection * connection = e.second;
@ -273,9 +272,8 @@ ParallelReplicas::ReplicaMap::iterator ParallelReplicas::waitForReadEvent()
read_list.push_back(connection->socket);
}
/** Если не было найдено никаких данных, то проверяем, есть ли соединения
* готовые для чтения.
*/
/// Если не было найдено никаких данных, то проверяем, есть ли соединения
/// готовые для чтения.
if (read_list.empty())
{
Poco::Net::Socket::SocketList write_list;
@ -287,9 +285,17 @@ ParallelReplicas::ReplicaMap::iterator ParallelReplicas::waitForReadEvent()
if (connection != nullptr)
read_list.push_back(connection->socket);
}
int n = Poco::Net::Socket::select(read_list, write_list, except_list, settings->poll_interval * 1000000);
int n = Poco::Net::Socket::select(read_list, write_list, except_list, settings->receive_timeout);
if (n == 0)
return replica_map.end();
{
std::stringstream description;
for (auto it = replica_map.begin(); it != replica_map.end(); ++it)
description << (it != replica_map.begin() ? ", " : "") << it->second->getDescription();
throw Exception("Timeout exceeded while reading from " + description.str(), ErrorCodes::TIMEOUT_EXCEEDED);
}
}
auto & socket = read_list[rand() % read_list.size()];

View File

@ -141,9 +141,12 @@ void Block::insertUnique(const ColumnWithTypeAndName & elem)
void Block::erase(size_t position)
{
if (index_by_position.empty())
throw Exception("Block is empty", ErrorCodes::POSITION_OUT_OF_BOUND);
if (position >= index_by_position.size())
throw Exception("Position out of bound in Block::erase(), max position = "
+ toString(index_by_position.size()), ErrorCodes::POSITION_OUT_OF_BOUND);
+ toString(index_by_position.size() - 1), ErrorCodes::POSITION_OUT_OF_BOUND);
Container_t::iterator it = index_by_position[position];
index_by_name.erase(index_by_name.find(it->name));
@ -177,6 +180,9 @@ void Block::erase(const String & name)
ColumnWithTypeAndName & Block::getByPosition(size_t position)
{
if (index_by_position.empty())
throw Exception("Block is empty", ErrorCodes::POSITION_OUT_OF_BOUND);
if (position >= index_by_position.size())
throw Exception("Position " + toString(position)
+ " is out of bound in Block::getByPosition(), max position = "
@ -189,6 +195,9 @@ ColumnWithTypeAndName & Block::getByPosition(size_t position)
const ColumnWithTypeAndName & Block::getByPosition(size_t position) const
{
if (index_by_position.empty())
throw Exception("Block is empty", ErrorCodes::POSITION_OUT_OF_BOUND);
if (position >= index_by_position.size())
throw Exception("Position " + toString(position)
+ " is out of bound in Block::getByPosition(), max position = "
@ -302,7 +311,13 @@ std::string Block::dumpStructure() const
{
if (it != data.begin())
res << ", ";
res << it->name << ' ' << it->type->getName() << ' ' << it->column->getName() << ' ' << it->column->size();
res << it->name << ' ' << it->type->getName();
if (it->column)
res << ' ' << it->column->getName() << ' ' << it->column->size();
else
res << " nullptr";
}
return res.str();
}

View File

@ -52,16 +52,16 @@ inline std::string demangle(const char * const mangled, int & status)
return demangled;
}
void tryLogCurrentException(const char * log_name)
void tryLogCurrentException(const char * log_name, const std::string & start_of_message)
{
tryLogCurrentException(&Logger::get(log_name));
tryLogCurrentException(&Logger::get(log_name), start_of_message);
}
void tryLogCurrentException(Poco::Logger * logger)
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message)
{
try
{
LOG_ERROR(logger, getCurrentExceptionMessage(true));
LOG_ERROR(logger, start_of_message << (start_of_message.empty() ? "" : ": ") << getCurrentExceptionMessage(true));
}
catch (...)
{

View File

@ -26,7 +26,7 @@ namespace DB
{
BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & buf,
Block & sample, size_t max_block_size) const
const Block & sample, size_t max_block_size) const
{
if (name == "Native")
return new NativeBlockInputStream(buf);
@ -48,7 +48,7 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu
BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & buf,
Block & sample) const
const Block & sample) const
{
if (name == "Native")
return new NativeBlockOutputStream(buf);

View File

@ -2,6 +2,7 @@
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/VarInt.h>
#include <DB/IO/CompressedReadBufferFromFile.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/DataTypes/DataTypeArray.h>
@ -14,6 +15,25 @@ namespace DB
{
NativeBlockInputStream::NativeBlockInputStream(
ReadBuffer & istr_, UInt64 server_revision_,
bool use_index_,
IndexForNativeFormat::Blocks::const_iterator index_block_it_,
IndexForNativeFormat::Blocks::const_iterator index_block_end_)
: istr(istr_), server_revision(server_revision_),
use_index(use_index_), index_block_it(index_block_it_), index_block_end(index_block_end_)
{
if (use_index)
{
istr_concrete = typeid_cast<CompressedReadBufferFromFile *>(&istr);
if (!istr_concrete)
throw Exception("When need to use index for NativeBlockInputStream, istr must be CompressedReadBufferFromFile.", ErrorCodes::LOGICAL_ERROR);
index_column_it = index_block_it->columns.begin();
}
}
void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows)
{
/** Для массивов требуется сначала десериализовать смещения, а потом значения.
@ -47,9 +67,17 @@ Block NativeBlockInputStream::readImpl()
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
if (istr.eof())
if (use_index && index_block_it == index_block_end)
return res;
if (istr.eof())
{
if (use_index)
throw Exception("Input doesn't contain all data for index.", ErrorCodes::CANNOT_READ_ALL_DATA);
return res;
}
/// Дополнительная информация о блоке.
if (server_revision >= DBMS_MIN_REVISION_WITH_BLOCK_INFO)
res.info.read(istr);
@ -57,29 +85,103 @@ Block NativeBlockInputStream::readImpl()
/// Размеры
size_t columns = 0;
size_t rows = 0;
readVarUInt(columns, istr);
readVarUInt(rows, istr);
if (!use_index)
{
readVarUInt(columns, istr);
readVarUInt(rows, istr);
}
else
{
columns = index_block_it->num_columns;
rows = index_block_it->num_rows;
}
for (size_t i = 0; i < columns; ++i)
{
if (use_index)
{
/// Если текущая позиция и так какая требуется, то реального seek-а не происходит.
istr_concrete->seek(index_column_it->location.offset_in_compressed_file, index_column_it->location.offset_in_decompressed_block);
}
ColumnWithTypeAndName column;
/// Имя
readStringBinary(column.name, istr);
readBinary(column.name, istr);
/// Тип
String type_name;
readStringBinary(type_name, istr);
readBinary(type_name, istr);
column.type = data_type_factory.get(type_name);
if (use_index)
{
/// Индекс позволяет сделать проверки.
if (index_column_it->name != column.name)
throw Exception("Index points to column with wrong name: corrupted index or data", ErrorCodes::INCORRECT_INDEX);
if (index_column_it->type != type_name)
throw Exception("Index points to column with wrong type: corrupted index or data", ErrorCodes::INCORRECT_INDEX);
}
/// Данные
column.column = column.type->createColumn();
readData(*column.type, *column.column, istr, rows);
res.insert(column);
if (use_index)
++index_column_it;
}
if (use_index)
{
if (index_column_it != index_block_it->columns.end())
throw Exception("Inconsistent index: not all columns were read", ErrorCodes::INCORRECT_INDEX);
++index_block_it;
if (index_block_it != index_block_end)
index_column_it = index_block_it->columns.begin();
}
return res;
}
void IndexForNativeFormat::read(ReadBuffer & istr, const NameSet & required_columns)
{
while (!istr.eof())
{
blocks.emplace_back();
IndexOfBlockForNativeFormat & block = blocks.back();
readVarUInt(block.num_columns, istr);
readVarUInt(block.num_rows, istr);
if (block.num_columns < required_columns.size())
throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX);
for (size_t i = 0; i < block.num_columns; ++i)
{
IndexOfOneColumnForNativeFormat column_index;
readBinary(column_index.name, istr);
readBinary(column_index.type, istr);
readBinary(column_index.location.offset_in_compressed_file, istr);
readBinary(column_index.location.offset_in_decompressed_block, istr);
if (required_columns.count(column_index.name))
block.columns.push_back(std::move(column_index));
}
if (block.columns.size() < required_columns.size())
throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX);
if (block.columns.size() > required_columns.size())
throw Exception("Index contain duplicate columns", ErrorCodes::INCORRECT_INDEX);
block.num_columns = block.columns.size();
}
}
}

View File

@ -2,12 +2,14 @@
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/VarInt.h>
#include <DB/IO/CompressedWriteBuffer.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataStreams/MarkInCompressedFile.h>
#include <DB/DataStreams/NativeBlockOutputStream.h>
@ -15,6 +17,21 @@ namespace DB
{
NativeBlockOutputStream::NativeBlockOutputStream(
WriteBuffer & ostr_, UInt64 client_revision_,
WriteBuffer * index_ostr_)
: ostr(ostr_), client_revision(client_revision_),
index_ostr(index_ostr_)
{
if (index_ostr)
{
ostr_concrete = typeid_cast<CompressedWriteBuffer *>(&ostr);
if (!ostr_concrete)
throw Exception("When need to write index for NativeBlockOutputStream, ostr must be CompressedWriteBuffer.", ErrorCodes::LOGICAL_ERROR);
}
}
void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit)
{
/** Если есть столбцы-константы - то материализуем их.
@ -71,11 +88,31 @@ void NativeBlockOutputStream::write(const Block & block)
/// Размеры
size_t columns = block.columns();
size_t rows = block.rows();
writeVarUInt(columns, ostr);
writeVarUInt(rows, ostr);
/** Индекс имеет ту же структуру, что и поток с данными.
* Но вместо значений столбца он содержит засечку, ссылающуюся на место в файле с данными, где находится этот кусочек столбца.
*/
if (index_ostr)
{
writeVarUInt(columns, *index_ostr);
writeVarUInt(rows, *index_ostr);
}
for (size_t i = 0; i < columns; ++i)
{
/// Для индекса.
MarkInCompressedFile mark;
if (index_ostr)
{
ostr_concrete->next(); /// Заканчиваем сжатый блок.
mark.offset_in_compressed_file = ostr_concrete->getCompressedBytes();
mark.offset_in_decompressed_block = ostr_concrete->getRemainingBytes();
}
const ColumnWithTypeAndName & column = block.getByPosition(i);
/// Имя
@ -86,6 +123,15 @@ void NativeBlockOutputStream::write(const Block & block)
/// Данные
writeData(*column.type, column.column, ostr, 0, 0);
if (index_ostr)
{
writeStringBinary(column.name, *index_ostr);
writeStringBinary(column.type->getName(), *index_ostr);
writeBinary(mark.offset_in_compressed_file, *index_ostr);
writeBinary(mark.offset_in_decompressed_block, *index_ostr);
}
}
}

View File

@ -137,12 +137,28 @@ void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr, size
ColumnString::Chars_t & data = column_string.getChars();
ColumnString::Offsets_t & offsets = column_string.getOffsets();
/// Выбрано наугад.
constexpr auto avg_value_size_hint_reserve_multiplier = 1.2;
double avg_chars_size;
double avg_chars_size = (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0])
? (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier
: DBMS_APPROX_STRING_SIZE);
if (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0]))
{
/// Выбрано наугад.
constexpr auto avg_value_size_hint_reserve_multiplier = 1.2;
avg_chars_size = (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier;
}
else
{
/** Небольшая эвристика для оценки того, что в столбце много пустых строк.
* В этом случае, для экономии оперативки, будем говорить, что средний размер значения маленький.
*/
if (istr.position() + sizeof(UInt32) <= istr.buffer().end()
&& *reinterpret_cast<const UInt32 *>(istr.position()) == 0) /// Первые 4 строки находятся в буфере и являются пустыми.
{
avg_chars_size = 1;
}
else
avg_chars_size = DBMS_APPROX_STRING_SIZE;
}
data.reserve(data.size() + std::ceil(limit * avg_chars_size));

View File

@ -27,6 +27,7 @@ void registerFunctionsArray(FunctionFactory & factory)
factory.registerFunction<FunctionEmptyArrayDate>();
factory.registerFunction<FunctionEmptyArrayDateTime>();
factory.registerFunction<FunctionEmptyArrayString>();
factory.registerFunction<FunctionEmptyArrayToSingle>();
factory.registerFunction<FunctionRange>();
}

View File

@ -9,6 +9,7 @@
#include <DB/DataTypes/DataTypeAggregateFunction.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/AggregateFunctions/AggregateFunctionCount.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Interpreters/Aggregator.h>
@ -1688,6 +1689,66 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
}
Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
{
if (blocks.empty())
return {};
StringRefs key(keys_size);
ConstColumnPlainPtrs key_columns(keys_size);
AggregateColumnsData aggregate_columns(aggregates_size);
initialize(blocks.front());
/// Каким способом выполнять агрегацию?
for (size_t i = 0; i < keys_size; ++i)
key_columns[i] = sample.getByPosition(i).column;
Sizes key_sizes;
AggregatedDataVariants::Type method = chooseAggregationMethod(key_columns, key_sizes);
/// Временные данные для агрегации.
AggregatedDataVariants result;
/// result будет уничтожать состояния агрегатных функций в деструкторе
result.aggregator = this;
result.init(method);
result.keys_size = keys_size;
result.key_sizes = key_sizes;
LOG_TRACE(log, "Merging partially aggregated blocks.");
for (Block & block : blocks)
{
if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows)
mergeWithoutKeyStreamsImpl(block, result);
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data);
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
else if (result.type != AggregatedDataVariants::Type::without_key)
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
BlocksList merged_block = convertToBlocks(result, final, 1);
if (merged_block.size() > 1) /// TODO overflows
throw Exception("Logical error: temporary result is not single-level", ErrorCodes::LOGICAL_ERROR);
LOG_TRACE(log, "Merged partially aggregated blocks.");
if (merged_block.empty())
return {};
return merged_block.front();
}
template <typename Method>
void NO_INLINE Aggregator::destroyImpl(
Method & method) const
@ -1769,4 +1830,5 @@ void Aggregator::setCancellationHook(const CancellationHook cancellation_hook)
isCancelled = cancellation_hook;
}
}

View File

@ -5,6 +5,7 @@
#include <Poco/SharedPtr.h>
#include <Poco/Mutex.h>
#include <Poco/File.h>
#include <Poco/UUIDGenerator.h>
#include <Yandex/logger_useful.h>
@ -96,6 +97,8 @@ struct ContextShared
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
Poco::SharedPtr<Clusters> clusters;
Poco::UUIDGenerator uuid_generator;
bool shutdown_called = false;
@ -587,8 +590,12 @@ void Context::setCurrentDatabase(const String & name)
void Context::setCurrentQueryId(const String & query_id)
{
String query_id_to_set = query_id;
if (query_id_to_set.empty()) /// Если пользователь не передал свой query_id, то генерируем его самостоятельно.
query_id_to_set = shared->uuid_generator.createRandom().toString();
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
current_query_id = query_id;
current_query_id = query_id_to_set;
}

View File

@ -31,6 +31,8 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
const auto & layout_type = keys.front();
if ("range_hashed" == layout_type)
@ -41,7 +43,7 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
ErrorCodes::BAD_ARGUMENTS
};
return std::make_unique<RangeHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
return std::make_unique<RangeHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}
else
{
@ -49,16 +51,15 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
throw Exception{
"Elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS
};
ErrorCodes::BAD_ARGUMENTS};
if ("flat" == layout_type)
{
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}
else if ("hashed" == layout_type)
{
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}
else if ("cache" == layout_type)
{
@ -66,8 +67,12 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
if (size == 0)
throw Exception{
"Dictionary of layout 'cache' cannot have 0 cells",
ErrorCodes::TOO_SMALL_BUFFER_SIZE
};
ErrorCodes::TOO_SMALL_BUFFER_SIZE};
if (require_nonempty)
throw Exception{
"Dictionary of layout 'cache' cannot have 'require_nonempty' attribute set",
ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
}

View File

@ -6,6 +6,7 @@
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Functions/IFunction.h>
#include <DB/Functions/FunctionsArray.h>
#include <set>
@ -268,6 +269,24 @@ void ExpressionAction::execute(Block & block) const
if (!any_array)
throw Exception("ARRAY JOIN of not array: " + *array_joined_columns.begin(), ErrorCodes::TYPE_MISMATCH);
/// Если LEFT ARRAY JOIN, то создаём столбцы, в которых пустые массивы заменены на массивы с одним элементом - значением по-умолчанию.
std::map<String, ColumnPtr> non_empty_array_columns;
if (array_join_is_left)
{
for (const auto & name : array_joined_columns)
{
auto src_col = block.getByName(name);
Block tmp_block{src_col, {{}, src_col.type, {}}};
FunctionEmptyArrayToSingle().execute(tmp_block, {0}, 1);
non_empty_array_columns[name] = tmp_block.getByPosition(1).column;
}
any_array_ptr = non_empty_array_columns.begin()->second;
any_array = typeid_cast<const ColumnArray *>(&*any_array_ptr);
}
size_t columns = block.columns();
for (size_t i = 0; i < columns; ++i)
{
@ -278,7 +297,8 @@ void ExpressionAction::execute(Block & block) const
if (!typeid_cast<const DataTypeArray *>(&*current.type))
throw Exception("ARRAY JOIN of not array: " + current.name, ErrorCodes::TYPE_MISMATCH);
ColumnPtr array_ptr = current.column;
ColumnPtr array_ptr = array_join_is_left ? non_empty_array_columns[current.name] : current.column;
if (array_ptr->isConst())
array_ptr = dynamic_cast<const IColumnConst &>(*array_ptr).convertToFullColumn();
@ -379,7 +399,7 @@ std::string ExpressionAction::toString() const
break;
case ARRAY_JOIN:
ss << "ARRAY JOIN ";
ss << (array_join_is_left ? "LEFT " : "") << "ARRAY JOIN ";
for (NameSet::const_iterator it = array_joined_columns.begin(); it != array_joined_columns.end(); ++it)
{
if (it != array_joined_columns.begin())
@ -761,7 +781,7 @@ std::string ExpressionActions::getID() const
ss << actions[i].result_name;
if (actions[i].type == ExpressionAction::ARRAY_JOIN)
{
ss << "{";
ss << (actions[i].array_join_is_left ? "LEFT ARRAY JOIN" : "ARRAY JOIN") << "{";
for (NameSet::const_iterator it = actions[i].array_joined_columns.begin();
it != actions[i].array_joined_columns.end(); ++it)
{

View File

@ -87,6 +87,18 @@ const std::unordered_set<String> possibly_injective_function_names
"dictGetDateTime"
};
static bool functionIsInOperator(const String & name)
{
return name == "in" || name == "notIn";
}
static bool functionIsInOrGlobalInOperator(const String & name)
{
return name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn";
}
void ExpressionAnalyzer::init()
{
select_query = typeid_cast<ASTSelectQuery *>(&*ast);
@ -95,6 +107,7 @@ void ExpressionAnalyzer::init()
LogicalExpressionsOptimizer logical_expressions_optimizer(select_query, settings);
logical_expressions_optimizer.optimizeDisjunctiveEqualityChains();
/// Добавляет в множество известных алиасов те, которые объявлены в структуре таблицы (ALIAS-столбцы).
addStorageAliases();
/// Создаёт словарь aliases: alias -> ASTPtr
@ -103,6 +116,9 @@ void ExpressionAnalyzer::init()
/// Common subexpression elimination. Rewrite rules.
normalizeTree();
/// Выполнение скалярных подзапросов - замена их на значения-константы.
executeScalarSubqueries();
/// GROUP BY injective function elimination.
optimizeGroupBy();
@ -223,9 +239,10 @@ void ExpressionAnalyzer::analyzeAggregation()
void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables()
{
/// Преобразует GLOBAL-подзапросы во внешние таблицы; кладёт их в словарь external_tables: name -> StoragePtr.
initGlobalSubqueries(ast);
/// Создаёт словарь external_tables: name -> StoragePtr.
/// Добавляет уже существующие внешние таблицы (не подзапросы) в словарь external_tables.
findExternalTables(ast);
}
@ -388,7 +405,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(
}
/// Может быть указано IN t, где t - таблица, что равносильно IN (SELECT * FROM t).
if (func_node->name == "in" || func_node->name == "notIn" || func_node->name == "globalIn" || func_node->name == "globalNotIn")
if (functionIsInOrGlobalInOperator(func_node->name))
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(&*func_node->arguments->children.at(1)))
right->kind = ASTIdentifier::Table;
@ -528,6 +545,145 @@ void ExpressionAnalyzer::normalizeTreeImpl(
finished_asts[initial_ast] = ast;
}
void ExpressionAnalyzer::executeScalarSubqueries()
{
if (!select_query)
executeScalarSubqueriesImpl(ast);
else
{
for (auto & child : ast->children)
{
/// Не опускаемся в FROM и JOIN.
if (child.get() != select_query->table.get() && child.get() != select_query->join.get())
executeScalarSubqueriesImpl(child);
}
}
}
static ASTPtr addTypeConversion(ASTLiteral * ast_, const String & type_name)
{
if (0 == type_name.compare(0, strlen("Array"), "Array"))
return ast_; /// Преобразование типов для массивов пока не поддерживаем.
auto ast = std::unique_ptr<ASTLiteral>(ast_);
ASTFunction * func = new ASTFunction(ast->range);
ASTPtr res = func;
func->alias = ast->alias;
ast->alias.clear();
func->kind = ASTFunction::FUNCTION;
func->name = "to" + type_name;
ASTExpressionList * exp_list = new ASTExpressionList(ast->range);
func->arguments = exp_list;
func->children.push_back(func->arguments);
exp_list->children.push_back(ast.release());
return res;
}
void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
{
/** Заменяем подзапросы, возвращающие ровно одну строку
* ("скалярные" подзапросы) на соответствующие константы.
*
* Если подзапрос возвращает более одного столбца, то он заменяется на кортеж констант.
*
* Особенности:
*
* Замена происходит во время анализа запроса, а не во время основной стадии выполнения.
* Это значит, что не будет работать индикатор прогресса во время выполнения этих запросов,
* а также такие запросы нельзя будет прервать.
*
* Зато результат запросов может быть использован для индекса в таблице.
*
* Скалярные подзапросы выполняются на сервере-инициаторе запроса.
* На удалённые серверы запрос отправляется с уже подставленными константами.
*/
if (ASTSubquery * subquery = typeid_cast<ASTSubquery *>(ast.get()))
{
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.limits.max_result_rows = 1;
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
ASTPtr query = subquery->children.at(0);
BlockIO res = InterpreterSelectQuery(query, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1).execute();
Block block;
try
{
block = res.in->read();
if (!block)
throw Exception("Scalar subquery returned empty result", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
if (block.rows() != 1 || res.in->read())
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
}
catch (const Exception & e)
{
if (e.code() == ErrorCodes::TOO_MUCH_ROWS)
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
else
throw;
}
size_t columns = block.columns();
if (columns == 1)
{
ASTLiteral * lit = new ASTLiteral(ast->range, (*block.getByPosition(0).column)[0]);
lit->alias = subquery->alias;
ast = addTypeConversion(lit, block.getByPosition(0).type->getName());
}
else
{
ASTFunction * tuple = new ASTFunction(ast->range);
tuple->alias = subquery->alias;
ast = tuple;
tuple->kind = ASTFunction::FUNCTION;
tuple->name = "tuple";
ASTExpressionList * exp_list = new ASTExpressionList(ast->range);
tuple->arguments = exp_list;
tuple->children.push_back(tuple->arguments);
exp_list->children.resize(columns);
for (size_t i = 0; i < columns; ++i)
{
exp_list->children[i] = addTypeConversion(
new ASTLiteral(ast->range, (*block.getByPosition(i).column)[0]),
block.getByPosition(i).type->getName());
}
}
}
else
{
/** Не опускаемся в подзапросы в аргументах IN.
* Но если аргумент - не подзапрос, то глубже внутри него могут быть подзапросы, и в них надо опускаться.
*/
ASTFunction * func = typeid_cast<ASTFunction *>(ast.get());
if (func && func->kind == ASTFunction::FUNCTION
&& functionIsInOrGlobalInOperator(func->name))
{
for (auto & child : ast->children)
{
if (child.get() != func->arguments)
executeScalarSubqueriesImpl(child);
else
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
if (i != 1 || !typeid_cast<ASTSubquery *>(func->arguments->children[i].get()))
executeScalarSubqueriesImpl(func->arguments->children[i]);
}
}
else
for (auto & child : ast->children)
executeScalarSubqueriesImpl(child);
}
}
void ExpressionAnalyzer::optimizeGroupBy()
{
if (!(select_query && select_query->group_expression_list))
@ -657,7 +813,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(ASTPtr & node, const Block & sampl
makeSetsForIndexImpl(child, sample_block);
ASTFunction * func = typeid_cast<ASTFunction *>(node.get());
if (func && func->kind == ASTFunction::FUNCTION && (func->name == "in" || func->name == "notIn"))
if (func && func->kind == ASTFunction::FUNCTION && functionIsInOperator(func->name))
{
IAST & args = *func->arguments;
ASTPtr & arg = args.children.at(1);
@ -693,7 +849,8 @@ static SharedPtr<InterpreterSelectQuery> interpretSubquery(
* Так как результат этого поздапроса - ещё не результат всего запроса.
* Вместо этого работают ограничения
* max_rows_in_set, max_bytes_in_set, set_overflow_mode,
* max_rows_in_join, max_bytes_in_join, join_overflow_mode.
* max_rows_in_join, max_bytes_in_join, join_overflow_mode,
* которые проверяются отдельно (в объектах Set, Join).
*/
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
@ -740,10 +897,6 @@ static SharedPtr<InterpreterSelectQuery> interpretSubquery(
void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
{
/// Сгенерируем имя для внешней таблицы.
while (context.tryGetExternalTable("_data" + toString(external_table_id)))
++external_table_id;
if (const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(&*subquery_or_table_name))
{
/// Если это уже внешняя таблица, ничего заполять не нужно. Просто запоминаем ее наличие.
@ -754,13 +907,20 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
}
}
/// Сгенерируем имя для внешней таблицы.
String external_table_name = "_data" + toString(external_table_id);
while (context.tryGetExternalTable(external_table_name)
|| external_tables.count(external_table_name))
{
++external_table_id;
external_table_name = "_data" + toString(external_table_id);
}
SharedPtr<InterpreterSelectQuery> interpreter = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1);
Block sample = interpreter->getSampleBlock();
NamesAndTypesListPtr columns = new NamesAndTypesList(sample.getColumnsList());
String external_table_name = "_data" + toString(external_table_id);
/** Заменяем подзапрос на имя временной таблицы.
* Именно в таком виде, запрос отправится на удалённый сервер.
* На удалённый сервер отправится эта временная таблица, и на его стороне,
@ -1213,7 +1373,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name));
NameSet joined_columns;
joined_columns.insert(result_name);
actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns));
actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false));
}
return;
@ -1221,7 +1381,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
if (node->kind == ASTFunction::FUNCTION)
{
if (node->name == "in" || node->name == "notIn" || node->name == "globalIn" || node->name == "globalNotIn")
if (functionIsInOrGlobalInOperator(node->name))
{
if (!no_subqueries)
{
@ -1510,7 +1670,7 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio
result_columns.insert(result_source.first);
}
actions->add(ExpressionAction::arrayJoin(result_columns));
actions->add(ExpressionAction::arrayJoin(result_columns, select_query->array_join_is_left));
}
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)

View File

@ -64,7 +64,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
try
{
auto dict_ptr = failed_dictionary.second.dict->clone();
if (dict_ptr->getCreationException())
if (const auto exception_ptr = dict_ptr->getCreationException())
{
/// recalculate next attempt time
std::uniform_int_distribution<std::uint64_t> distribution(
@ -72,10 +72,11 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
failed_dictionary.second.next_attempt_time = std::chrono::system_clock::now() +
std::chrono::seconds{
std::min<std::uint64_t>(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine))
};
std::min<std::uint64_t>(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine))};
++failed_dictionary.second.error_count;
std::rethrow_exception(exception_ptr);
}
else
{
@ -99,7 +100,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
}
catch (...)
{
LOG_ERROR(log, "Failed reloading " << name << " dictionary due to unexpected error");
tryLogCurrentException(log, "Failed reloading '" + name + "' dictionary");
}
}
@ -114,6 +115,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
try
{
/// Если словарь не удалось ни разу загрузить или даже не удалось инициализировать из конфига.
if (!dictionary.second.dict)
continue;
@ -144,6 +146,10 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
{
/// create new version of dictionary
auto new_version = current->clone();
if (const auto exception_ptr = new_version->getCreationException())
std::rethrow_exception(exception_ptr);
dictionary.second.dict->set(new_version.release());
}
}
@ -155,25 +161,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
{
dictionary.second.exception = std::current_exception();
try
{
throw;
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "Cannot update external dictionary '" << name
<< "'! You must resolve this manually. " << e.displayText());
}
catch (const std::exception & e)
{
LOG_ERROR(log, "Cannot update external dictionary '" << name
<< "'! You must resolve this manually. " << e.what());
}
catch (...)
{
LOG_ERROR(log, "Cannot update external dictionary '" << name
<< "'! You must resolve this manually.");
}
tryLogCurrentException(log, "Cannot update external dictionary '" + name + "', leaving old version");
}
}
}
@ -235,6 +223,8 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
throw std::runtime_error{"Overriding dictionary from file " + dict_it->second.origin};
auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context);
/// Если словарь не удалось загрузить.
if (const auto exception_ptr = dict_ptr->getCreationException())
{
const auto failed_dict_it = failed_dictionaries.find(name);
@ -292,6 +282,9 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
{
if (!name.empty())
{
/// Если для словаря не удалось загрузить данные или даже не удалось инициализировать из конфига.
/// - всё-равно вставляем информацию в dictionaries, с нулевым указателем dict.
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
const auto exception_ptr = std::current_exception();
@ -302,25 +295,7 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
dict_it->second.exception = exception_ptr;
}
try
{
throw;
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name
<< "'! You must resolve this manually. " << e.displayText());
}
catch (const std::exception & e)
{
LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name
<< "'! You must resolve this manually. " << e.what());
}
catch (...)
{
LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name
<< "'! You must resolve this manually.");
}
tryLogCurrentException(log, "Cannot create external dictionary '" + name + "' from config path " + config_path);
/// propagate exception
if (throw_on_error)

View File

@ -7,7 +7,7 @@
#include <DB/IO/WriteHelpers.h>
#include <DB/DataStreams/MaterializingBlockInputStream.h>
#include <DB/DataStreams/copyData.h>
#include <DB/DataStreams/NullAndDoCopyBlockInputStream.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTNameTypePair.h>
@ -42,7 +42,7 @@ InterpreterCreateQuery::InterpreterCreateQuery(ASTPtr query_ptr_, Context & cont
}
void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists)
BlockIO InterpreterCreateQuery::executeImpl(bool assume_metadata_exists)
{
String path = context.getPath();
String current_database = context.getCurrentDatabase();
@ -81,7 +81,7 @@ void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists)
if (!create.if_not_exists || !context.isDatabaseExist(database_name))
context.addDatabase(database_name);
return;
return {};
}
SharedPtr<InterpreterSelectQuery> interpreter_select;
@ -118,7 +118,7 @@ void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists)
if (context.isTableExist(database_name, table_name))
{
if (create.if_not_exists)
return;
return {};
else
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
@ -251,9 +251,16 @@ void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists)
/// Если запрос CREATE SELECT, то вставим в таблицу данные
if (create.select && storage_name != "View" && (storage_name != "MaterializedView" || create.is_populate))
{
BlockInputStreamPtr from = new MaterializingBlockInputStream(interpreter_select->execute().in);
copyData(*from, *res->write(query_ptr));
BlockIO io;
io.in_sample = select_sample;
io.in = new NullAndDoCopyBlockInputStream(
new MaterializingBlockInputStream(interpreter_select->execute().in),
res->write(query_ptr));
return io;
}
return {};
}
InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(ASTPtr expression_list)

View File

@ -100,6 +100,7 @@ BlockIO InterpreterInsertQuery::execute()
InterpreterSelectQuery interpreter_select{query.select, context};
BlockInputStreamPtr in{interpreter_select.execute().in};
res.in = new NullAndDoCopyBlockInputStream{in, out};
res.in_sample = interpreter_select.getSampleBlock();
}
return res;

View File

@ -6,6 +6,7 @@
#include <DB/DataStreams/MergingSortedBlockInputStream.h>
#include <DB/DataStreams/AggregatingBlockInputStream.h>
#include <DB/DataStreams/MergingAggregatedBlockInputStream.h>
#include <DB/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h>
#include <DB/DataStreams/AsynchronousBlockInputStream.h>
#include <DB/DataStreams/UnionBlockInputStream.h>
#include <DB/DataStreams/ParallelAggregatingBlockInputStream.h>
@ -330,9 +331,6 @@ BlockIO InterpreterSelectQuery::execute()
/// Ограничения на результат, квота на результат, а также колбек для прогресса.
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(&*streams[0]))
{
stream->setProgressCallback(context.getProgressCallback());
stream->setProcessListElement(context.getProcessListElement());
/// Ограничения действуют только на конечный результат.
if (to_stage == QueryProcessingStage::Complete)
{
@ -406,7 +404,7 @@ void InterpreterSelectQuery::executeSingleQuery()
bool has_having = false;
bool has_order_by = false;
ExpressionActionsPtr before_join;
ExpressionActionsPtr before_join; /// включая JOIN
ExpressionActionsPtr before_where;
ExpressionActionsPtr before_aggregation;
ExpressionActionsPtr before_having;
@ -859,14 +857,38 @@ void InterpreterSelectQuery::executeAggregation(ExpressionActionsPtr expression,
void InterpreterSelectQuery::executeMergeAggregated(bool overflow_row, bool final)
{
/// Склеим несколько источников в один
executeUnion();
/// Теперь объединим агрегированные блоки
Names key_names;
AggregateDescriptions aggregates;
query_analyzer->getAggregateInfo(key_names, aggregates);
streams[0] = new MergingAggregatedBlockInputStream(streams[0], key_names, aggregates, overflow_row, final, original_max_threads);
/** Есть два режима распределённой агрегации.
*
* 1. В разных потоках читать из удалённых серверов блоки.
* Сохранить все блоки в оперативку. Объединить блоки.
* Если агрегация двухуровневая - распараллелить по номерам корзин.
*
* 2. В одном потоке читать по очереди блоки с разных серверов.
* В оперативке хранится только по одному блоку с каждого сервера.
* Если агрегация двухуровневая - последовательно объединяем блоки каждого следующего уровня.
*
* Второй вариант расходует меньше памяти (до 256 раз меньше)
* в случае двухуровневой агрегации, которая используется для больших результатов после GROUP BY,
* но при этом может работать медленнее.
*/
if (!settings.distributed_aggregation_memory_efficient)
{
/// Склеим несколько источников в один, распараллеливая работу.
executeUnion();
/// Теперь объединим агрегированные блоки
streams[0] = new MergingAggregatedBlockInputStream(streams[0], key_names, aggregates, overflow_row, final, original_max_threads);
}
else
{
streams[0] = new MergingAggregatedMemoryEfficientBlockInputStream(streams, key_names, aggregates, overflow_row, final);
streams.resize(1);
}
}

View File

@ -166,6 +166,15 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Держим элемент списка процессов до конца обработки запроса.
res.process_list_entry = process_list_entry;
if (res.in)
{
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(res.in.get()))
{
stream->setProgressCallback(context.getProgressCallback());
stream->setProcessListElement(context.getProcessListElement());
}
}
quota.addQuery(current_time);
/// Всё, что связано с логом запросов.

View File

@ -41,7 +41,7 @@ static void executeCreateQuery(const String & query, Context & context, const St
{
if (const auto id = dynamic_cast<const ASTFunction *>(ast_create_query.storage.get()))
{
if (id->name == "TinyLog")
if (id->name == "TinyLog" || id->name == "StripeLog")
{
tryLogCurrentException(__PRETTY_FUNCTION__);
return;

View File

@ -147,28 +147,38 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit)
}
void stableSortBlock(Block & block, const SortDescription & description)
void stableGetPermutation(const Block & block, const SortDescription & description, IColumn::Permutation & out_permutation)
{
if (!block)
return;
size_t size = block.rows();
IColumn::Permutation perm(size);
out_permutation.resize(size);
for (size_t i = 0; i < size; ++i)
perm[i] = i;
out_permutation[i] = i;
ColumnsWithSortDescriptions columns_with_sort_desc;
for (size_t i = 0, size = description.size(); i < size; ++i)
{
IColumn * column = !description[i].column_name.empty()
const IColumn * column = !description[i].column_name.empty()
? block.getByName(description[i].column_name).column
: block.getByPosition(description[i].column_number).column;
columns_with_sort_desc.push_back(std::make_pair(column, description[i]));
}
std::stable_sort(perm.begin(), perm.end(), PartialSortingLess(columns_with_sort_desc));
std::stable_sort(out_permutation.begin(), out_permutation.end(), PartialSortingLess(columns_with_sort_desc));
}
void stableSortBlock(Block & block, const SortDescription & description)
{
if (!block)
return;
IColumn::Permutation perm;
stableGetPermutation(block, description, perm);
size_t columns = block.columns();
for (size_t i = 0; i < columns; ++i)

Some files were not shown because too many files have changed in this diff Show More