This commit is contained in:
Roman Peshkurov 2015-08-13 15:35:57 +03:00
commit f4c87c238f
61 changed files with 2027 additions and 1287 deletions

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

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

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

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

@ -283,6 +283,8 @@ namespace ErrorCodes
INDEX_NOT_USED = 277,
LEADERSHIP_LOST = 278,
ALL_CONNECTION_TRIES_FAILED = 279,
NO_AVAILABLE_DATA = 280,
DICTIONARY_IS_EMPTY = 281,
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

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

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

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

@ -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,39 @@ 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 : "");
FormatStateStacked frame_with_indent = frame;
++frame_with_indent.indent;
table->formatImpl(settings, state, frame_with_indent);
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

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

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

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

@ -317,7 +317,7 @@ std::string Block::dumpStructure() const
if (it->column)
res << ' ' << it->column->getName() << ' ' << it->column->size();
else
res << "nullptr";
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

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

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

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

View File

@ -0,0 +1,186 @@
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTFunction.h>
namespace DB
{
void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
FormatStateStacked nested_need_parens = frame;
FormatStateStacked nested_dont_need_parens = frame;
nested_need_parens.need_parens = true;
nested_dont_need_parens.need_parens = false;
/// Стоит ли записать эту функцию в виде оператора?
bool written = false;
if (arguments && !parameters)
{
if (arguments->children.size() == 1)
{
const char * operators[] =
{
"negate", "-",
"not", "NOT ",
nullptr
};
for (const char ** func = operators; *func; func += 2)
{
if (0 == strcmp(name.c_str(), func[0]))
{
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
/** Особо дурацкий случай. Если у нас унарный минус перед литералом, являющимся отрицательным числом:
* "-(-1)" или "- -1", то это нельзя форматировать как --1, так как это будет воспринято как комментарий.
* Вместо этого, добавим пробел.
* PS. Нельзя просто попросить добавить скобки - см. formatImpl для ASTLiteral.
*/
if (name == "negate" && typeid_cast<const ASTLiteral *>(&*arguments->children[0]))
settings.ostr << ' ';
arguments->formatImpl(settings, state, nested_need_parens);
written = true;
}
}
}
/** need_parens - нужны ли скобки вокруг выражения с оператором.
* Они нужны, только если это выражение входит в другое выражение с оператором.
*/
if (!written && arguments->children.size() == 2)
{
const char * operators[] =
{
"multiply", " * ",
"divide", " / ",
"modulo", " % ",
"plus", " + ",
"minus", " - ",
"notEquals", " != ",
"lessOrEquals", " <= ",
"greaterOrEquals", " >= ",
"less", " < ",
"greater", " > ",
"equals", " = ",
"like", " LIKE ",
"notLike", " NOT LIKE ",
"in", " IN ",
"notIn", " NOT IN ",
"globalIn", " GLOBAL IN ",
"globalNotIn", " GLOBAL NOT IN ",
nullptr
};
for (const char ** func = operators; *func; func += 2)
{
if (0 == strcmp(name.c_str(), func[0]))
{
if (frame.need_parens)
settings.ostr << '(';
arguments->children[0]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(settings, state, nested_need_parens);
if (frame.need_parens)
settings.ostr << ')';
written = true;
}
}
if (!written && 0 == strcmp(name.c_str(), "arrayElement"))
{
arguments->children[0]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : "");
written = true;
}
if (!written && 0 == strcmp(name.c_str(), "tupleElement"))
{
arguments->children[0]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << "." << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(settings, state, nested_need_parens);
written = true;
}
}
if (!written && arguments->children.size() >= 2)
{
const char * operators[] =
{
"and", " AND ",
"or", " OR ",
nullptr
};
for (const char ** func = operators; *func; func += 2)
{
if (0 == strcmp(name.c_str(), func[0]))
{
if (frame.need_parens)
settings.ostr << '(';
for (size_t i = 0; i < arguments->children.size(); ++i)
{
if (i != 0)
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
arguments->children[i]->formatImpl(settings, state, nested_need_parens);
}
if (frame.need_parens)
settings.ostr << ')';
written = true;
}
}
}
if (!written && arguments->children.size() >= 1 && 0 == strcmp(name.c_str(), "array"))
{
settings.ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : "");
for (size_t i = 0; i < arguments->children.size(); ++i)
{
if (i != 0)
settings.ostr << ", ";
arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens);
}
settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : "");
written = true;
}
if (!written && arguments->children.size() >= 2 && 0 == strcmp(name.c_str(), "tuple"))
{
settings.ostr << (settings.hilite ? hilite_operator : "") << '(' << (settings.hilite ? hilite_none : "");
for (size_t i = 0; i < arguments->children.size(); ++i)
{
if (i != 0)
settings.ostr << ", ";
arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens);
}
settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : "");
written = true;
}
}
if (!written)
{
settings.ostr << (settings.hilite ? hilite_function : "") << name;
if (parameters)
{
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
parameters->formatImpl(settings, state, nested_dont_need_parens);
settings.ostr << (settings.hilite ? hilite_function : "") << ')';
}
if (arguments)
{
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
arguments->formatImpl(settings, state, nested_dont_need_parens);
settings.ostr << (settings.hilite ? hilite_function : "") << ')';
}
settings.ostr << (settings.hilite ? hilite_none : "");
}
}
}

View File

@ -1,5 +1,7 @@
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTSelectQuery.h>
namespace DB
{
@ -217,5 +219,154 @@ const IAST * ASTSelectQuery::getFormat() const
return query->format.get();
}
void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
{
frame.current_select = this;
frame.need_parens = false;
std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' ');
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (distinct ? "DISTINCT " : "") << (s.hilite ? hilite_none : "");
s.one_line
? select_expression_list->formatImpl(s, state, frame)
: typeid_cast<const ASTExpressionList &>(*select_expression_list).formatImplMultiline(s, state, frame);
if (table)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FROM " << (s.hilite ? hilite_none : "");
if (database)
{
database->formatImpl(s, state, frame);
s.ostr << ".";
}
if (typeid_cast<const ASTSelectQuery *>(&*table))
{
if (s.one_line)
s.ostr << " (";
else
s.ostr << "\n" << indent_str << "(\n";
FormatStateStacked frame_with_indent = frame;
++frame_with_indent.indent;
table->formatImpl(s, state, frame_with_indent);
if (s.one_line)
s.ostr << ")";
else
s.ostr << "\n" << indent_str << ")";
}
else
table->formatImpl(s, state, frame);
}
if (final)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FINAL" << (s.hilite ? hilite_none : "");
}
if (sample_size)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SAMPLE " << (s.hilite ? hilite_none : "");
sample_size->formatImpl(s, state, frame);
}
if (array_join_expression_list)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str
<< (array_join_is_left ? "LEFT " : "") << "ARRAY JOIN " << (s.hilite ? hilite_none : "");
s.one_line
? array_join_expression_list->formatImpl(s, state, frame)
: typeid_cast<const ASTExpressionList &>(*array_join_expression_list).formatImplMultiline(s, state, frame);
}
if (join)
{
s.ostr << " ";
join->formatImpl(s, state, frame);
}
if (prewhere_expression)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "PREWHERE " << (s.hilite ? hilite_none : "");
prewhere_expression->formatImpl(s, state, frame);
}
if (where_expression)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "WHERE " << (s.hilite ? hilite_none : "");
where_expression->formatImpl(s, state, frame);
}
if (group_expression_list)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : "");
s.one_line
? group_expression_list->formatImpl(s, state, frame)
: typeid_cast<const ASTExpressionList &>(*group_expression_list).formatImplMultiline(s, state, frame);
}
if (group_by_with_totals)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : "");
if (having_expression)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "HAVING " << (s.hilite ? hilite_none : "");
having_expression->formatImpl(s, state, frame);
}
if (order_expression_list)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : "");
s.one_line
? order_expression_list->formatImpl(s, state, frame)
: typeid_cast<const ASTExpressionList &>(*order_expression_list).formatImplMultiline(s, state, frame);
}
if (limit_length)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "LIMIT " << (s.hilite ? hilite_none : "");
if (limit_offset)
{
limit_offset->formatImpl(s, state, frame);
s.ostr << ", ";
}
limit_length->formatImpl(s, state, frame);
}
if (settings)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SETTINGS " << (s.hilite ? hilite_none : "");
const ASTSetQuery & ast_set = typeid_cast<const ASTSetQuery &>(*settings);
for (ASTSetQuery::Changes::const_iterator it = ast_set.changes.begin(); it != ast_set.changes.end(); ++it)
{
if (it != ast_set.changes.begin())
s.ostr << ", ";
s.ostr << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value);
}
}
if (format)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FORMAT " << (s.hilite ? hilite_none : "");
format->formatImpl(s, state, frame);
}
if (next_union_all)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "UNION ALL " << s.nl_or_ws << (s.hilite ? hilite_none : "");
// NOTE Мы можем безопасно применить static_cast вместо typeid_cast, потому что знаем, что в цепочке UNION ALL
// имеются только деревья типа SELECT.
const ASTSelectQuery & next_ast = static_cast<const ASTSelectQuery &>(*next_union_all);
next_ast.formatImpl(s, state, frame);
}
}
};

View File

@ -0,0 +1,33 @@
#include <DB/Parsers/ASTWithAlias.h>
namespace DB
{
void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (!alias.empty())
{
/// Если мы уже ранее вывели этот узел в другом месте запроса, то теперь достаточно вывести лишь алиас.
if (!state.printed_asts_with_alias.emplace(frame.current_select, alias).second)
{
WriteBufferFromOStream wb(settings.ostr, 32);
writeProbablyBackQuotedString(alias, wb);
return;
}
}
/// Если есть алиас, то требуются скобки вокруг всего выражения, включая алиас. Потому что запись вида 0 AS x + 0 синтаксически некорректна.
if (frame.need_parens && !alias.empty())
settings.ostr <<'(';
formatImplWithoutAlias(settings, state, frame);
if (!alias.empty())
{
writeAlias(alias, settings.ostr, settings.hilite);
if (frame.need_parens)
settings.ostr <<')';
}
}
}

39
dbms/src/Parsers/IAST.cpp Normal file
View File

@ -0,0 +1,39 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Parsers/IAST.h>
namespace DB
{
const char * IAST::hilite_keyword = "\033[1m";
const char * IAST::hilite_identifier = "\033[0;36m";
const char * IAST::hilite_function = "\033[0;33m";
const char * IAST::hilite_operator = "\033[1;33m";
const char * IAST::hilite_alias = "\033[0;32m";
const char * IAST::hilite_none = "\033[0m";
/// Квотировать идентификатор обратными кавычками, если это требуется.
String backQuoteIfNeed(const String & x)
{
String res(x.size(), '\0');
{
WriteBufferFromString wb(res);
writeProbablyBackQuotedString(x, wb);
}
return res;
}
void IAST::writeAlias(const String & name, std::ostream & s, bool hilite) const
{
s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_alias : "");
WriteBufferFromOStream wb(s, 32);
writeProbablyBackQuotedString(name, wb);
wb.next();
s << (hilite ? hilite_none : "");
}
}

View File

@ -1,957 +1,9 @@
#include <sstream>
#include <mysqlxx/Manip.h>
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTDropQuery.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTColumnDeclaration.h>
#include <DB/Parsers/ASTAsterisk.h>
#include <DB/Parsers/ASTOrderByElement.h>
#include <DB/Parsers/ASTSubquery.h>
#include <DB/Parsers/ASTAlterQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Parsers/ASTJoin.h>
#include <DB/Parsers/ASTCheckQuery.h>
//#include <DB/Parsers/ASTMultiQuery.h>
#include <DB/Parsers/formatAST.h>
namespace DB
{
static const char * hilite_keyword = "\033[1m";
static const char * hilite_identifier = "\033[0;36m";
static const char * hilite_function = "\033[0;33m";
static const char * hilite_operator = "\033[1;33m";
static const char * hilite_alias = "\033[0;32m";
static const char * hilite_none = "\033[0m";
/// Квотировать идентификатор обратными кавычками, если это требуется.
String backQuoteIfNeed(const String & x)
{
String res(x.size(), '\0');
{
WriteBufferFromString wb(res);
writeProbablyBackQuotedString(x, wb);
}
return res;
}
static String hightlight(const String & keyword, const String & color_sequence, const bool hilite)
{
return hilite ? color_sequence + keyword + hilite_none : keyword;
}
static void writeAlias(const String & name, std::ostream & s, bool hilite, bool one_line)
{
s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_alias : "");
WriteBufferFromOStream wb(s, 32);
writeProbablyBackQuotedString(name, wb);
wb.next();
s << (hilite ? hilite_none : "");
}
void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it)
{
if (it != ast.children.begin())
s << ", ";
formatAST(**it, s, indent, hilite, one_line, need_parens);
}
}
/** Вывести список выражений в секциях запроса SELECT - по одному выражению на строку.
*/
static void formatExpressionListMultiline(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite)
{
std::string indent_str = "\n" + std::string(4 * (indent + 1), ' ');
for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it)
{
if (it != ast.children.begin())
s << ", ";
if (ast.children.size() > 1)
s << indent_str;
formatAST(**it, s, indent + 1, hilite, false);
}
}
void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
std::string nl_or_nothing = one_line ? "" : "\n";
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
std::string nl_or_ws = one_line ? " " : "\n";
s << (hilite ? hilite_keyword : "") << indent_str << "SELECT " << (ast.distinct ? "DISTINCT " : "") << (hilite ? hilite_none : "");
one_line
? formatAST(*ast.select_expression_list, s, indent, hilite, one_line)
: formatExpressionListMultiline(typeid_cast<const ASTExpressionList &>(*ast.select_expression_list), s, indent, hilite);
if (ast.table)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FROM " << (hilite ? hilite_none : "");
if (ast.database)
{
formatAST(*ast.database, s, indent, hilite, one_line);
s << ".";
}
if (typeid_cast<const ASTSelectQuery *>(&*ast.table))
{
if (one_line)
s << " (";
else
s << "\n" << indent_str << "(\n";
formatAST(*ast.table, s, indent + 1, hilite, one_line);
if (one_line)
s << ")";
else
s << "\n" << indent_str << ")";
}
else
formatAST(*ast.table, s, indent, hilite, one_line);
}
if (ast.final)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FINAL" << (hilite ? hilite_none : "");
}
if (ast.sample_size)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "SAMPLE " << (hilite ? hilite_none : "");
formatAST(*ast.sample_size, s, indent, hilite, one_line);
}
if (ast.array_join_expression_list)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str
<< (ast.array_join_is_left ? "LEFT " : "") << "ARRAY JOIN " << (hilite ? hilite_none : "");
one_line
? formatAST(*ast.array_join_expression_list, s, indent, hilite, one_line)
: formatExpressionListMultiline(typeid_cast<const ASTExpressionList &>(*ast.array_join_expression_list), s, indent, hilite);
}
if (ast.join)
{
s << " ";
formatAST(*ast.join, s, indent, hilite, one_line);
}
if (ast.prewhere_expression)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "PREWHERE " << (hilite ? hilite_none : "");
formatAST(*ast.prewhere_expression, s, indent, hilite, one_line);
}
if (ast.where_expression)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "WHERE " << (hilite ? hilite_none : "");
formatAST(*ast.where_expression, s, indent, hilite, one_line);
}
if (ast.group_expression_list)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "GROUP BY " << (hilite ? hilite_none : "");
one_line
? formatAST(*ast.group_expression_list, s, indent, hilite, one_line)
: formatExpressionListMultiline(typeid_cast<const ASTExpressionList &>(*ast.group_expression_list), s, indent, hilite);
}
if (ast.group_by_with_totals)
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << (one_line ? "" : " ") << "WITH TOTALS" << (hilite ? hilite_none : "");
if (ast.having_expression)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "HAVING " << (hilite ? hilite_none : "");
formatAST(*ast.having_expression, s, indent, hilite, one_line);
}
if (ast.order_expression_list)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "ORDER BY " << (hilite ? hilite_none : "");
one_line
? formatAST(*ast.order_expression_list, s, indent, hilite, one_line)
: formatExpressionListMultiline(typeid_cast<const ASTExpressionList &>(*ast.order_expression_list), s, indent, hilite);
}
if (ast.limit_length)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "LIMIT " << (hilite ? hilite_none : "");
if (ast.limit_offset)
{
formatAST(*ast.limit_offset, s, indent, hilite, one_line);
s << ", ";
}
formatAST(*ast.limit_length, s, indent, hilite, one_line);
}
if (ast.settings)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "SETTINGS " << (hilite ? hilite_none : "");
const ASTSetQuery & ast_set = typeid_cast<const ASTSetQuery &>(*ast.settings);
for (ASTSetQuery::Changes::const_iterator it = ast_set.changes.begin(); it != ast_set.changes.end(); ++it)
{
if (it != ast_set.changes.begin())
s << ", ";
s << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value);
}
}
if (ast.format)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FORMAT " << (hilite ? hilite_none : "");
formatAST(*ast.format, s, indent, hilite, one_line);
}
if (ast.next_union_all)
{
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "UNION ALL " << nl_or_ws << (hilite ? hilite_none : "");
// NOTE Мы можем безопасно применить static_cast вместо typeid_cast, потому что знаем, что в цепочке UNION ALL
// имеются только деревья типа SELECT.
const ASTSelectQuery & next_ast = static_cast<const ASTSelectQuery &>(*ast.next_union_all);
formatAST(next_ast, s, indent, hilite, one_line, need_parens);
}
}
void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
/// Если есть алиас, то требуются скобки вокруг всего выражения, включая алиас. Потому что запись вида 0 AS x + 0 синтаксически некорректна.
if (need_parens && !ast.alias.empty())
s << '(';
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
std::string nl_or_nothing = one_line ? "" : "\n";
s << nl_or_nothing << indent_str << "(" << nl_or_nothing;
formatAST(*ast.children[0], s, indent + 1, hilite, one_line);
s << nl_or_nothing << indent_str << ")";
if (!ast.alias.empty())
{
writeAlias(ast.alias, s, hilite, one_line);
if (need_parens)
s << ')';
}
}
void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
std::string nl_or_ws = one_line ? " " : "\n";
if (!ast.database.empty() && ast.table.empty())
{
s << (hilite ? hilite_keyword : "") << (ast.attach ? "ATTACH DATABASE " : "CREATE DATABASE ") << (ast.if_not_exists ? "IF NOT EXISTS " : "") << (hilite ? hilite_none : "")
<< backQuoteIfNeed(ast.database);
return;
}
{
std::string what = "TABLE";
if (ast.is_view)
what = "VIEW";
if (ast.is_materialized_view)
what = "MATERIALIZED VIEW";
s << (hilite ? hilite_keyword : "") << (ast.attach ? "ATTACH " : "CREATE ") << (ast.is_temporary ? "TEMPORARY " : "") << what << " " << (ast.if_not_exists ? "IF NOT EXISTS " : "") << (hilite ? hilite_none : "")
<< (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table);
}
if (!ast.as_table.empty())
{
s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_none : "")
<< (!ast.as_database.empty() ? backQuoteIfNeed(ast.as_database) + "." : "") << backQuoteIfNeed(ast.as_table);
}
if (ast.columns)
{
s << (one_line ? " (" : "\n(");
formatAST(*ast.columns, s, indent + 1, hilite, one_line);
s << (one_line ? ")" : "\n)");
}
if (ast.storage && !ast.is_materialized_view && !ast.is_view)
{
s << (hilite ? hilite_keyword : "") << " ENGINE" << (hilite ? hilite_none : "") << " = ";
formatAST(*ast.storage, s, indent, hilite, one_line);
}
if (ast.inner_storage)
{
s << (hilite ? hilite_keyword : "") << " ENGINE" << (hilite ? hilite_none : "") << " = ";
formatAST(*ast.inner_storage, s, indent, hilite, one_line);
}
if (ast.is_populate)
{
s << (hilite ? hilite_keyword : "") << " POPULATE" << (hilite ? hilite_none : "");
}
if (ast.select)
{
s << (hilite ? hilite_keyword : "") << " AS" << nl_or_ws << (hilite ? hilite_none : "");
formatAST(*ast.select, s, indent, hilite, one_line);
}
}
void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
if (ast.table.empty() && !ast.database.empty())
{
s << (hilite ? hilite_keyword : "") << (ast.detach ? "DETACH DATABASE " : "DROP DATABASE ") << (ast.if_exists ? "IF EXISTS " : "") << (hilite ? hilite_none : "") << backQuoteIfNeed(ast.database);
return;
}
s << (hilite ? hilite_keyword : "") << (ast.detach ? "DETACH TABLE " : "DROP TABLE ") << (ast.if_exists ? "IF EXISTS " : "") << (hilite ? hilite_none : "")
<< (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table);
}
void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (hilite ? hilite_none : "")
<< (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table);
}
void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "") << name << " " << (hilite ? hilite_none : "")
<< (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table);
if (ast.format)
{
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
std::string nl_or_ws = one_line ? " " : "\n";
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FORMAT " << (hilite ? hilite_none : "");
formatAST(*ast.format, s, indent, hilite, one_line);
}
}
void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "EXISTS TABLE", s, indent, hilite, one_line, false);
}
void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "DESCRIBE TABLE", s, indent, hilite, one_line, false);
}
void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "SHOW CREATE TABLE", s, indent, hilite, one_line, false);
}
void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "") << "RENAME TABLE " << (hilite ? hilite_none : "");
for (ASTRenameQuery::Elements::const_iterator it = ast.elements.begin(); it != ast.elements.end(); ++it)
{
if (it != ast.elements.begin())
s << ", ";
s << (!it->from.database.empty() ? backQuoteIfNeed(it->from.database) + "." : "") << backQuoteIfNeed(it->from.table)
<< (hilite ? hilite_keyword : "") << " TO " << (hilite ? hilite_none : "")
<< (!it->to.database.empty() ? backQuoteIfNeed(it->to.database) + "." : "") << backQuoteIfNeed(it->to.table);
}
}
void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "") << "SET " << (ast.global ? "GLOBAL " : "") << (hilite ? hilite_none : "");
for (ASTSetQuery::Changes::const_iterator it = ast.changes.begin(); it != ast.changes.end(); ++it)
{
if (it != ast.changes.begin())
s << ", ";
s << it->name << " = " << apply_visitor(FieldVisitorToString(), it->value);
}
}
void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
if (ast.databases)
{
s << (hilite ? hilite_keyword : "") << "SHOW DATABASES" << (hilite ? hilite_none : "");
}
else
{
s << (hilite ? hilite_keyword : "") << "SHOW TABLES" << (hilite ? hilite_none : "");
if (!ast.from.empty())
s << (hilite ? hilite_keyword : "") << " FROM " << (hilite ? hilite_none : "")
<< backQuoteIfNeed(ast.from);
if (!ast.like.empty())
s << (hilite ? hilite_keyword : "") << " LIKE " << (hilite ? hilite_none : "")
<< mysqlxx::quote << ast.like;
}
if (ast.format)
{
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
std::string nl_or_ws = one_line ? " " : "\n";
s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "FORMAT " << (hilite ? hilite_none : "");
formatAST(*ast.format, s, indent, hilite, one_line);
}
}
void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "") << "USE " << (hilite ? hilite_none : "") << backQuoteIfNeed(ast.database);
return;
}
void formatAST(const ASTShowProcesslistQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "") << "SHOW PROCESSLIST" << (hilite ? hilite_none : "");
return;
}
void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "") << "INSERT INTO " << (hilite ? hilite_none : "")
<< (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table);
if (!ast.insert_id.empty())
s << (hilite ? hilite_keyword : "") << " ID = " << (hilite ? hilite_none : "")
<< mysqlxx::quote << ast.insert_id;
if (ast.columns)
{
s << " (";
formatAST(*ast.columns, s, indent, hilite, one_line);
s << ")";
}
if (ast.select)
{
s << " ";
formatAST(*ast.select, s, indent, hilite, one_line);
}
else
{
if (!ast.format.empty())
{
s << (hilite ? hilite_keyword : "") << " FORMAT " << (hilite ? hilite_none : "") << ast.format;
}
else
{
s << (hilite ? hilite_keyword : "") << " VALUES" << (hilite ? hilite_none : "");
}
}
}
void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
/// Если есть алиас, то требуются скобки вокруг всего выражения, включая алиас. Потому что запись вида 0 AS x + 0 синтаксически некорректна.
if (need_parens && !ast.alias.empty())
s << '(';
/// Стоит ли записать эту функцию в виде оператора?
bool written = false;
if (ast.arguments && !ast.parameters)
{
if (ast.arguments->children.size() == 1)
{
const char * operators[] =
{
"negate", "-",
"not", "NOT ",
nullptr
};
for (const char ** func = operators; *func; func += 2)
{
if (0 == strcmp(ast.name.c_str(), func[0]))
{
s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : "");
/** Особо дурацкий случай. Если у нас унарный минус перед литералом, являющимся отрицательным числом:
* "-(-1)" или "- -1", то это нельзя форматировать как --1, так как это будет воспринято как комментарий.
* Вместо этого, добавим пробел.
* PS. Нельзя просто попросить добавить скобки - см. formatAST для ASTLiteral.
*/
if (ast.name == "negate" && typeid_cast<const ASTLiteral *>(&*ast.arguments->children[0]))
s << ' ';
formatAST(*ast.arguments, s, indent, hilite, one_line, true);
written = true;
}
}
}
/** need_parens - нужны ли скобки вокруг выражения с оператором.
* Они нужны, только если это выражение входит в другое выражение с оператором.
*/
if (!written && ast.arguments->children.size() == 2)
{
const char * operators[] =
{
"multiply", " * ",
"divide", " / ",
"modulo", " % ",
"plus", " + ",
"minus", " - ",
"notEquals", " != ",
"lessOrEquals", " <= ",
"greaterOrEquals", " >= ",
"less", " < ",
"greater", " > ",
"equals", " = ",
"like", " LIKE ",
"notLike", " NOT LIKE ",
"in", " IN ",
"notIn", " NOT IN ",
"globalIn", " GLOBAL IN ",
"globalNotIn", " GLOBAL NOT IN ",
nullptr
};
for (const char ** func = operators; *func; func += 2)
{
if (0 == strcmp(ast.name.c_str(), func[0]))
{
if (need_parens)
s << '(';
formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true);
s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : "");
formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true);
if (need_parens)
s << ')';
written = true;
}
}
if (!written && 0 == strcmp(ast.name.c_str(), "arrayElement"))
{
formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true);
s << (hilite ? hilite_operator : "") << '[' << (hilite ? hilite_none : "");
formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true);
s << (hilite ? hilite_operator : "") << ']' << (hilite ? hilite_none : "");
written = true;
}
if (!written && 0 == strcmp(ast.name.c_str(), "tupleElement"))
{
formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true);
s << (hilite ? hilite_operator : "") << "." << (hilite ? hilite_none : "");
formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true);
written = true;
}
}
if (!written && ast.arguments->children.size() >= 2)
{
const char * operators[] =
{
"and", " AND ",
"or", " OR ",
nullptr
};
for (const char ** func = operators; *func; func += 2)
{
if (0 == strcmp(ast.name.c_str(), func[0]))
{
if (need_parens)
s << '(';
for (size_t i = 0; i < ast.arguments->children.size(); ++i)
{
if (i != 0)
s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : "");
formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, true);
}
if (need_parens)
s << ')';
written = true;
}
}
}
if (!written && ast.arguments->children.size() >= 1 && 0 == strcmp(ast.name.c_str(), "array"))
{
s << (hilite ? hilite_operator : "") << '[' << (hilite ? hilite_none : "");
for (size_t i = 0; i < ast.arguments->children.size(); ++i)
{
if (i != 0)
s << ", ";
formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, false);
}
s << (hilite ? hilite_operator : "") << ']' << (hilite ? hilite_none : "");
written = true;
}
if (!written && ast.arguments->children.size() >= 2 && 0 == strcmp(ast.name.c_str(), "tuple"))
{
s << (hilite ? hilite_operator : "") << '(' << (hilite ? hilite_none : "");
for (size_t i = 0; i < ast.arguments->children.size(); ++i)
{
if (i != 0)
s << ", ";
formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, false);
}
s << (hilite ? hilite_operator : "") << ')' << (hilite ? hilite_none : "");
written = true;
}
}
if (!written)
{
s << (hilite ? hilite_function : "") << ast.name;
if (ast.parameters)
{
s << '(' << (hilite ? hilite_none : "");
formatAST(*ast.parameters, s, indent, hilite, one_line);
s << (hilite ? hilite_function : "") << ')';
}
if (ast.arguments)
{
s << '(' << (hilite ? hilite_none : "");
formatAST(*ast.arguments, s, indent, hilite, one_line);
s << (hilite ? hilite_function : "") << ')';
}
s << (hilite ? hilite_none : "");
}
if (!ast.alias.empty())
{
writeAlias(ast.alias, s, hilite, one_line);
if (need_parens)
s << ')';
}
}
void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
if (need_parens && !ast.alias.empty())
s << '(';
s << (hilite ? hilite_identifier : "");
WriteBufferFromOStream wb(s, 32);
writeProbablyBackQuotedString(ast.name, wb);
wb.next();
s << (hilite ? hilite_none : "");
if (!ast.alias.empty())
{
writeAlias(ast.alias, s, hilite, one_line);
if (need_parens)
s << ')';
}
}
void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
if (need_parens && !ast.alias.empty())
s << '(';
s << apply_visitor(FieldVisitorToString(), ast.value);
if (!ast.alias.empty())
{
writeAlias(ast.alias, s, hilite, one_line);
if (need_parens)
s << ')';
}
}
void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
std::string nl_or_ws = one_line ? " " : "\n";
s << nl_or_ws << indent_str << backQuoteIfNeed(ast.name) << " ";
formatAST(*ast.type, s, indent, hilite, one_line);
}
void formatAST(const ASTColumnDeclaration & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
std::string nl_or_ws = one_line ? " " : "\n";
s << nl_or_ws << indent_str << backQuoteIfNeed(ast.name);
if (ast.type)
{
s << ' ';
formatAST(*ast.type, s, indent, hilite, one_line);
}
if (ast.default_expression)
{
s << ' ' << hightlight(ast.default_specifier, hilite_keyword, hilite) << ' ';
formatAST(*ast.default_expression, s, indent, hilite, one_line);
}
}
void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << "*";
}
void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
formatAST(*ast.children.front(), s, indent, hilite, one_line);
s << (hilite ? hilite_keyword : "") << (ast.direction == -1 ? " DESC" : " ASC") << (hilite ? hilite_none : "");
if (!ast.collator.isNull())
{
s << (hilite ? hilite_keyword : "") << " COLLATE " << (hilite ? hilite_none : "")
<< "'" << ast.collator->getLocale() << "'";
}
}
void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
std::string nl_or_nothing = one_line ? "" : "\n";
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
std::string nl_or_ws = one_line ? " " : "\n";
s << (hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (hilite ? hilite_none : "");
if (!ast.table.empty())
{
if (!ast.database.empty())
{
s << indent_str << ast.database;
s << ".";
}
s << indent_str << ast.table;
}
s << nl_or_ws;
for (size_t i = 0; i < ast.parameters.size(); ++i)
{
const ASTAlterQuery::Parameters &p = ast.parameters[i];
if (p.type == ASTAlterQuery::ADD_COLUMN)
{
s << (hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (hilite ? hilite_none : "");
formatAST(*p.col_decl, s, indent, hilite, true);
/// AFTER
if (p.column)
{
s << (hilite ? hilite_keyword : "") << indent_str << " AFTER " << (hilite ? hilite_none : "");
formatAST(*p.column, s, indent, hilite, one_line);
}
}
else if (p.type == ASTAlterQuery::DROP_COLUMN)
{
s << (hilite ? hilite_keyword : "") << indent_str << "DROP COLUMN " << (hilite ? hilite_none : "");
formatAST(*p.column, s, indent, hilite, true);
}
else if (p.type == ASTAlterQuery::MODIFY_COLUMN)
{
s << (hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (hilite ? hilite_none : "");
formatAST(*p.col_decl, s, indent, hilite, true);
}
else if (p.type == ASTAlterQuery::DROP_PARTITION)
{
s << (hilite ? hilite_keyword : "") << indent_str << (p.detach ? "DETACH" : "DROP") << " PARTITION "
<< (hilite ? hilite_none : "");
formatAST(*p.partition, s, indent, hilite, true);
}
else if (p.type == ASTAlterQuery::ATTACH_PARTITION)
{
s << (hilite ? hilite_keyword : "") << indent_str << "ATTACH " << (p.unreplicated ? "UNREPLICATED " : "")
<< (p.part ? "PART " : "PARTITION ") << (hilite ? hilite_none : "");
formatAST(*p.partition, s, indent, hilite, true);
}
else if (p.type == ASTAlterQuery::FETCH_PARTITION)
{
s << (hilite ? hilite_keyword : "") << indent_str << "FETCH " << (p.unreplicated ? "UNREPLICATED " : "")
<< "PARTITION " << (hilite ? hilite_none : "");
formatAST(*p.partition, s, indent, hilite, true);
s << (hilite ? hilite_keyword : "") << " FROM " << (hilite ? hilite_none : "")
<< mysqlxx::quote << p.from;
}
else if (p.type == ASTAlterQuery::FREEZE_PARTITION)
{
s << (hilite ? hilite_keyword : "") << indent_str << "FREEZE PARTITION " << (hilite ? hilite_none : "");
formatAST(*p.partition, s, indent, hilite, true);
}
else
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
std::string comma = (i < (ast.parameters.size() -1) ) ? "," : "";
s << (hilite ? hilite_keyword : "") << indent_str << comma << (hilite ? hilite_none : "");
s << nl_or_ws;
}
}
void formatAST(const ASTSet & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
/** Подготовленное множество. В пользовательских запросах такого не бывает, но такое бывает после промежуточных преобразований запроса.
* Выведем его не по-настоящему (это не будет корректным запросом, но покажет, что здесь было множество).
*/
s << (hilite ? hilite_keyword : "")
<< "(...)"
<< (hilite ? hilite_none : "");
}
void formatAST(const ASTJoin & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "");
if (ast.locality == ASTJoin::Global)
s << "GLOBAL ";
if (ast.kind != ASTJoin::Cross)
s << (ast.strictness == ASTJoin::Any ? "ANY " : "ALL ");
s << (ast.kind == ASTJoin::Inner ? "INNER "
: (ast.kind == ASTJoin::Left ? "LEFT "
: (ast.kind == ASTJoin::Right ? "RIGHT "
: (ast.kind == ASTJoin::Cross ? "CROSS "
: "FULL OUTER "))));
s << "JOIN "
<< (hilite ? hilite_none : "");
formatAST(*ast.table, s, indent, hilite, one_line, need_parens);
if (ast.kind != ASTJoin::Cross)
{
s << (hilite ? hilite_keyword : "") << " USING " << (hilite ? hilite_none : "");
formatAST(*ast.using_expr_list, s, indent, hilite, one_line, need_parens);
}
}
void formatAST(const ASTCheckQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
std::string nl_or_nothing = one_line ? "" : "\n";
std::string indent_str = one_line ? "" : std::string(4 * indent, ' ');
std::string nl_or_ws = one_line ? " " : "\n";
s << (hilite ? hilite_keyword : "") << indent_str << "CHECK TABLE " << (hilite ? hilite_none : "");
if (!ast.table.empty())
{
if (!ast.database.empty())
{
s << (hilite ? hilite_keyword : "") << indent_str << ast.database << (hilite ? hilite_none : "");
s << ".";
}
s << (hilite ? hilite_keyword : "") << indent_str << ast.table << (hilite ? hilite_none : "");
}
s << nl_or_ws;
}
/*
void formatAST(const ASTMultiQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
s << (hilite ? hilite_keyword : "") << "{" << (hilite ? hilite_none : "");
for (const auto & child : ast.children)
{
s << "\n";
formatAST(*child, s, indent + 1, hilite, one_line, need_parens);
s << ";\n";
}
s << (hilite ? hilite_keyword : "") << "}" << (hilite ? hilite_none : "");
}*/
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
#define DISPATCH(NAME) \
else if (const AST ## NAME * concrete = typeid_cast<const AST ## NAME *>(&ast)) \
formatAST(*concrete, s, indent, hilite, one_line, need_parens);
if (false) {}
DISPATCH(SelectQuery)
DISPATCH(InsertQuery)
DISPATCH(CreateQuery)
DISPATCH(DropQuery)
DISPATCH(RenameQuery)
DISPATCH(ShowTablesQuery)
DISPATCH(UseQuery)
DISPATCH(SetQuery)
DISPATCH(OptimizeQuery)
DISPATCH(ExistsQuery)
DISPATCH(ShowCreateQuery)
DISPATCH(DescribeQuery)
DISPATCH(ExpressionList)
DISPATCH(Function)
DISPATCH(Identifier)
DISPATCH(Literal)
DISPATCH(NameTypePair)
DISPATCH(ColumnDeclaration)
DISPATCH(Asterisk)
DISPATCH(OrderByElement)
DISPATCH(Subquery)
DISPATCH(AlterQuery)
DISPATCH(ShowProcesslistQuery)
DISPATCH(Set)
DISPATCH(Join)
DISPATCH(CheckQuery)
// DISPATCH(MultiQuery)
else
throw Exception("Unknown element in AST: " + ast.getID()
+ ((ast.range.first && (ast.range.second > ast.range.first))
? " '" + std::string(ast.range.first, ast.range.second - ast.range.first) + "'"
: ""),
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
#undef DISPATCH
}
String formatColumnsForCreateQuery(NamesAndTypesList & columns)
{
std::string res;

View File

@ -485,15 +485,12 @@ int Server::main(const std::vector<std::string> & args)
if (config().has("interserver_http_port"))
{
String this_host;
if (config().has("interserver_http_host"))
{
this_host = config().getString("interserver_http_host");
}
else
String this_host = config().getString("interserver_http_host", "");
if (this_host.empty())
{
this_host = getFQDNOrHostName();
LOG_DEBUG(log, "Configuration parameter 'interserver_http_host' doesn't exist. Will use '" + this_host + "' as replica host.");
LOG_DEBUG(log, "Configuration parameter 'interserver_http_host' doesn't exist or exists and empty. Will use '" + this_host + "' as replica host.");
}
String port_str = config().getString("interserver_http_port");

View File

@ -0,0 +1,3 @@
0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
1 1 (2,2)
1 1 (2,2)

View File

@ -0,0 +1,8 @@
SELECT toUInt64(1) IN (1234567890, 2345678901, 3456789012, 4567890123, 5678901234, 6789012345, 7890123456, 8901234567, 9012345678, 123456789) AS x,
x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x,
x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x,
x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x,
x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x, x
FROM remote('127.0.0.1', system, one) SETTINGS max_query_size = 10000;
SELECT 1 AS x, x, (SELECT 2 AS x, x) FROM remote('127.0.0.{1,2}', system.one) WHERE (3, 4) IN (SELECT 3 AS x, toUInt8(x + 1));

View File

@ -0,0 +1,416 @@
1 1
3 1
6 1
7 1
9 1
11 1
14 1
17 1
19 1
20 2
26 1
31 1
35 1
36 1
0 159
1 164
3 165
6 162
7 160
9 164
10 81
11 158
13 161
14 160
17 163
19 164
20 159
21 161
22 159
26 160
31 164
35 160
36 161
0 54571
1 55013
3 52912
6 52353
7 54011
9 54138
10 26870
11 54554
13 53951
14 53396
17 55227
19 55115
20 54370
21 54268
22 54620
26 53394
31 54151
35 54328
36 52997
0.125 1
0.5 1
0.05 1
0.143 1
0.056 1
0.048 2
0.083 1
0.25 1
0.1 1
0.028 1
0.027 1
0.031 1
0.067 1
0.037 1
0.045 161
0.125 160
0.5 164
0.05 164
0.143 162
0.091 81
0.056 163
0.048 159
0.083 158
0.25 165
1 159
0.1 164
0.028 160
0.027 161
0.031 164
0.067 160
0.043 159
0.037 160
0.071 161
0.045 54268
0.125 54011
0.5 55013
0.05 55115
0.143 52353
0.091 26870
0.056 55227
0.048 54370
0.083 54554
0.25 52912
1 54571
0.1 54138
0.028 54328
0.027 52997
0.031 54151
0.067 53396
0.043 54620
0.037 53394
0.071 53951
0.5 1
0.05 1
0.25 1
0.048 2
0.083 1
0.125 1
0.031 1
0.143 1
0.028 1
0.067 1
0.027 1
0.056 1
0.037 1
0.1 1
0.5 164
0.05 164
0.25 165
0.048 159
0.091 81
0.043 159
0.071 161
0.083 158
0.125 160
0.031 164
0.143 162
0.028 160
0.067 160
0.045 161
0.027 161
0.056 163
0.037 160
0.1 164
1 159
0.5 55013
0.05 55115
0.25 52912
0.048 54370
0.091 26870
0.043 54620
0.071 53951
0.083 54554
0.125 54011
0.031 54151
0.143 52353
0.028 54328
0.067 53396
0.045 54268
0.027 52997
0.056 55227
0.037 53394
0.1 54138
1 54571
1 1
3 1
6 1
7 1
9 1
11 1
14 1
17 1
19 1
20 2
26 1
31 1
35 1
36 1
0 162
1 158
3 162
6 163
7 162
9 162
10 79
11 162
13 163
14 160
17 163
19 158
20 162
21 157
22 164
26 162
31 161
35 162
36 163
0 54029
1 53772
3 53540
6 54012
7 53910
9 52761
10 26462
11 52701
13 54505
14 53790
17 54064
19 55420
20 56686
21 52639
22 54251
26 53827
31 53574
35 55022
36 53961
1 1
3 1
6 1
7 1
9 1
11 1
14 1
17 1
19 1
20 2
26 1
31 1
35 1
36 1
0 162
1 162
3 162
6 162
7 163
9 163
10 81
11 163
13 162
14 162
17 162
19 162
20 162
21 162
22 162
26 162
31 162
35 162
36 162
0 54226
1 54034
3 54016
6 53982
7 54076
9 54218
10 27075
11 54093
13 54108
14 54096
17 54294
19 54070
20 54028
21 54170
22 54106
26 54103
31 54050
35 54130
36 53868
0.125 1
0.5 1
0.05 1
0.143 1
0.056 1
0.048 2
0.083 1
0.25 1
0.1 1
0.028 1
0.027 1
0.031 1
0.067 1
0.037 1
0.045 162
0.125 163
0.5 162
0.05 162
0.143 162
0.091 81
0.056 162
0.048 162
0.083 163
0.25 162
1 162
0.1 163
0.028 162
0.027 162
0.031 162
0.067 162
0.043 162
0.037 162
0.071 162
0.045 54170
0.125 54076
0.5 54034
0.05 54070
0.143 53982
0.091 27075
0.056 54294
0.048 54028
0.083 54093
0.25 54016
1 54226
0.1 54218
0.028 54130
0.027 53868
0.031 54050
0.067 54096
0.043 54106
0.037 54103
0.071 54108
0.5 1
0.05 1
0.25 1
0.048 2
0.083 1
0.125 1
0.031 1
0.143 1
0.028 1
0.067 1
0.027 1
0.056 1
0.037 1
0.1 1
0.5 162
0.05 162
0.25 162
0.048 162
0.091 81
0.043 162
0.071 162
0.083 163
0.125 163
0.031 162
0.143 162
0.028 162
0.067 162
0.045 162
0.027 162
0.056 162
0.037 162
0.1 163
1 162
0.5 54034
0.05 54070
0.25 54016
0.048 54028
0.091 27075
0.043 54106
0.071 54108
0.083 54093
0.125 54076
0.031 54050
0.143 53982
0.028 54130
0.067 54096
0.045 54170
0.027 53868
0.056 54294
0.037 54103
0.1 54218
1 54226
1 1
3 1
6 1
7 1
9 1
11 1
14 1
17 1
19 1
20 2
26 1
31 1
35 1
36 1
0 162
1 162
3 162
6 162
7 163
9 163
10 81
11 163
13 162
14 162
17 162
19 162
20 162
21 162
22 162
26 162
31 162
35 162
36 162
0 54195
1 54086
3 54127
6 54173
7 53969
9 54257
10 26985
11 53880
13 54105
14 54043
17 54176
19 53913
20 54088
21 53991
22 54112
26 54136
31 54074
35 54153
36 53999

View File

@ -0,0 +1,35 @@
/* uniqHLL12 */
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqHLL12(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqHLL12(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
/* uniqCombined */
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y;
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y;
SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y;

View File

@ -13,10 +13,8 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & cfg
cfg.keys(config_name, replica_keys);
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = replica_keys.begin(); it != replica_keys.end(); ++it)
{
if (!(*it == "port" || *it == "user" || *it == "password" || *it == "db" || *it == "table"))
if (*it == "replica") /// На том же уровне могут быть другие параметры.
{
if (it->size() < std::string("replica").size() || it->substr(0, std::string("replica").size()) != "replica")
throw Poco::Exception("Unknown element in config: " + *it + ", expected replica");
std::string replica_name = config_name + "." + *it;
Replica replica(new Pool(cfg, replica_name, default_connections, max_connections, config_name.c_str()),
cfg.getInt(replica_name + ".priority", 0));