This commit is contained in:
Roman Peshkurov 2015-07-02 15:59:54 +03:00
commit d5e7f57b61
80 changed files with 3035 additions and 609 deletions

View File

@ -8,6 +8,7 @@ Download HP Vertica 7.1.1 Analytic Database Server, Debian or Ubuntu 14.04 versi
sudo apt-get install sysstat pstack mcelog
sudo dpkg -i vertica_7.1.1-0_amd64.deb
sudo sh -c "echo 'export TZ=Europe/Moscow' >> /home/dbadmin/.bash_profile"
# Don't specify localhost due to poor support of IPv6.
sudo /opt/vertica/sbin/install_vertica --hosts=127.0.0.1 --failure-threshold=NONE
sudo mkdir /opt/vertica-data/
sudo chown dbadmin /opt/vertica-data/
@ -23,8 +24,6 @@ both directories: /opt/vertica-data/
main menu
exit
PS. Note that Vertica doesn't support IPv6.
How to prepare data
-------------------

View File

@ -26,13 +26,13 @@ class IConnectionPool : private boost::noncopyable
{
public:
typedef PoolBase<Connection>::Entry Entry;
virtual Entry get(Settings * settings = nullptr) = 0;
virtual Entry get(const Settings * settings = nullptr) = 0;
/** Выделяет до указанного количества соединений для работы.
* Соединения предоставляют доступ к разным репликам одного шарда.
* Выкидывает исключение, если не удалось выделить ни одного соединения.
*/
virtual std::vector<Entry> getMany(Settings * settings = nullptr)
virtual std::vector<Entry> getMany(const Settings * settings = nullptr)
{
return std::vector<Entry>{ get(settings) };
}
@ -89,7 +89,7 @@ public:
/** Выделяет соединение для работы. */
Entry get(Settings * settings = nullptr) override
Entry get(const Settings * settings = nullptr) override
{
if (settings)
return Base::get(settings->queue_max_wait_ms.totalMilliseconds());

View File

@ -21,11 +21,11 @@ namespace DB
*
* Замечание: если один из вложенных пулов заблокируется из-за переполнения, то этот пул тоже заблокируется.
*/
class ConnectionPoolWithFailover : public PoolWithFailoverBase<IConnectionPool, Settings *>, public IConnectionPool
class ConnectionPoolWithFailover : public PoolWithFailoverBase<IConnectionPool>, public IConnectionPool
{
public:
typedef IConnectionPool::Entry Entry;
typedef PoolWithFailoverBase<IConnectionPool, Settings *> Base;
typedef PoolWithFailoverBase<IConnectionPool> Base;
ConnectionPoolWithFailover(ConnectionPools & nested_pools_,
LoadBalancing load_balancing,
@ -52,7 +52,7 @@ public:
}
/** Выделяет соединение для работы. */
Entry get(Settings * settings = nullptr) override
Entry get(const Settings * settings = nullptr) override
{
applyLoadBalancing(settings);
return Base::get(settings);
@ -61,14 +61,14 @@ public:
/** Выделяет до указанного количества соединений для работы.
* Соединения предоставляют доступ к разным репликам одного шарда.
*/
std::vector<Entry> getMany(Settings * settings = nullptr) override
std::vector<Entry> getMany(const Settings * settings = nullptr) override
{
applyLoadBalancing(settings);
return Base::getMany(settings);
}
protected:
bool tryGet(ConnectionPoolPtr pool, Settings * settings, Entry & out_entry, std::stringstream & fail_message) override
bool tryGet(ConnectionPoolPtr pool, const Settings * settings, Entry & out_entry, std::stringstream & fail_message) override
{
try
{
@ -90,7 +90,7 @@ private:
std::vector<size_t> hostname_differences; /// Расстояния от имени этого хоста до имен хостов пулов.
LoadBalancing default_load_balancing;
void applyLoadBalancing(Settings * settings)
void applyLoadBalancing(const Settings * settings)
{
LoadBalancing load_balancing = default_load_balancing;
if (settings)

View File

@ -21,10 +21,10 @@ class ParallelReplicas final : private boost::noncopyable
{
public:
/// Принимает готовое соединение.
ParallelReplicas(Connection * connection_, Settings * settings_, ThrottlerPtr throttler_);
ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_);
/// Принимает пул, из которого нужно будет достать одно или несколько соединений.
ParallelReplicas(IConnectionPool * pool_, Settings * settings_, ThrottlerPtr throttler_);
ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_);
/// Отправить на реплики всё содержимое внешних таблиц.
void sendExternalTablesData(std::vector<ExternalTablesData> & data);
@ -82,7 +82,7 @@ private:
void invalidateReplica(ReplicaMap::iterator it);
private:
Settings * settings;
const Settings * settings;
ReplicaMap replica_map;
/// Если не nullptr, то используется, чтобы ограничить сетевой трафик.

View File

@ -24,36 +24,51 @@ public:
};
/** шаблон для столбцов-констант (столбцов одинаковых значений).
/** Столбец-константа может содержать внутри себя само значение,
* или, в случае массивов, SharedPtr от значения-массива,
* чтобы избежать проблем производительности при копировании очень больших массивов.
*
* T - тип значения,
* DataHolder - как значение хранится в таблице (либо T, либо SharedPtr<T>)
* Derived должен реализовать методы getDataFromHolderImpl - получить ссылку на значение из holder-а.
*
* Для строк и массивов реализации sizeOfField и byteSize могут быть некорректными.
*/
template <typename T>
class ColumnConst final : public IColumnConst
template <typename T, typename DataHolder, typename Derived>
class ColumnConstBase : public IColumnConst
{
protected:
size_t s;
DataHolder data;
DataTypePtr data_type;
T & getDataFromHolder() { return static_cast<Derived *>(this)->getDataFromHolderImpl(); }
const T & getDataFromHolder() const { return static_cast<const Derived *>(this)->getDataFromHolderImpl(); }
ColumnConstBase(size_t s_, const DataHolder & data_, DataTypePtr data_type_)
: s(s_), data(data_), data_type(data_type_) {}
public:
typedef T Type;
typedef typename NearestFieldType<T>::Type FieldType;
/// Для ColumnConst<Array> data_type_ должен быть ненулевым.
/// Для ColumnConst<String> data_type_ должен быть ненулевым, если тип данных FixedString.
ColumnConst(size_t s_, const T & data_, DataTypePtr data_type_ = DataTypePtr()) : s(s_), data(data_), data_type(data_type_) {}
std::string getName() const override { return "ColumnConst<" + TypeName<T>::get() + ">"; }
bool isNumeric() const override { return IsNumber<T>::value; }
bool isFixed() const override { return IsNumber<T>::value; }
size_t sizeOfField() const override { return sizeof(T); }
ColumnPtr cloneResized(size_t s_) const override { return new ColumnConst(s_, data, data_type); }
ColumnPtr cloneResized(size_t s_) const override { return new Derived(s_, data, data_type); }
size_t size() const override { return s; }
Field operator[](size_t n) const override { return FieldType(data); }
void get(size_t n, Field & res) const override { res = FieldType(data); }
Field operator[](size_t n) const override { return FieldType(getDataFromHolder()); }
void get(size_t n, Field & res) const override { res = FieldType(getDataFromHolder()); }
ColumnPtr cut(size_t start, size_t length) const override
{
return new ColumnConst(length, data, data_type);
return new Derived(length, data, data_type);
}
void insert(const Field & x) override
{
if (x.get<FieldType>() != FieldType(data))
if (x.get<FieldType>() != FieldType(getDataFromHolder()))
throw Exception("Cannot insert different element into constant column " + getName(),
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
++s;
@ -66,7 +81,7 @@ public:
void insertFrom(const IColumn & src, size_t n) override
{
if (data != static_cast<const ColumnConst<T> &>(src).data)
if (getDataFromHolder() != static_cast<const Derived &>(src).getDataFromHolder())
throw Exception("Cannot insert different element into constant column " + getName(),
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
++s;
@ -79,7 +94,7 @@ public:
if (s != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
return new ColumnConst(countBytesInFilter(filt), data, data_type);
return new Derived(countBytesInFilter(filt), data, data_type);
}
ColumnPtr replicate(const Offsets_t & offsets) const override
@ -88,7 +103,7 @@ public:
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
size_t replicated_size = 0 == s ? 0 : offsets.back();
return new ColumnConst(replicated_size, data, data_type);
return new Derived(replicated_size, data, data_type);
}
size_t byteSize() const override { return sizeof(data) + sizeof(s); }
@ -103,13 +118,13 @@ public:
if (perm.size() < limit)
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
return new ColumnConst(limit, data, data_type);
return new Derived(limit, data, data_type);
}
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
{
const ColumnConst<T> & rhs = static_cast<const ColumnConst<T> &>(rhs_);
return data < rhs.data /// TODO: правильное сравнение NaN-ов в константных столбцах.
const Derived & rhs = static_cast<const Derived &>(rhs_);
return getDataFromHolder() < rhs.getDataFromHolder() /// TODO: правильное сравнение NaN-ов в константных столбцах.
? -1
: (data == rhs.data
? 0
@ -123,30 +138,79 @@ public:
res[i] = i;
}
DataTypePtr & getDataType() { return data_type; }
const DataTypePtr & getDataType() const { return data_type; }
};
/** шаблон для столбцов-констант (столбцов одинаковых значений).
*/
template <typename T>
class ColumnConst final : public ColumnConstBase<T, T, ColumnConst<T>>
{
private:
friend class ColumnConstBase<T, T, ColumnConst<T>>;
T & getDataFromHolderImpl() { return this->data; }
const T & getDataFromHolderImpl() const { return this->data; }
public:
/// Для ColumnConst<Array> data_type_ должен быть ненулевым.
/// Для ColumnConst<String> data_type_ должен быть ненулевым, если тип данных FixedString.
ColumnConst(size_t s_, const T & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<T, T, ColumnConst<T>>(s_, data_, data_type_) {}
StringRef getDataAt(size_t n) const override;
StringRef getDataAtWithTerminatingZero(size_t n) const override;
UInt64 get64(size_t n) const override;
/** Более эффективные методы манипуляции */
T & getData() { return data; }
const T & getData() const { return data; }
T & getData() { return this->data; }
const T & getData() const { return this->data; }
/** Преобразование из константы в полноценный столбец */
ColumnPtr convertToFullColumn() const override;
void getExtremes(Field & min, Field & max) const override
{
min = FieldType(data);
max = FieldType(data);
min = typename ColumnConstBase<T, T, ColumnConst<T>>::FieldType(this->data);
max = typename ColumnConstBase<T, T, ColumnConst<T>>::FieldType(this->data);
}
};
DataTypePtr & getDataType() { return data_type; }
const DataTypePtr & getDataType() const { return data_type; }
template <>
class ColumnConst<Array> final : public ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>
{
private:
size_t s;
T data;
DataTypePtr data_type;
friend class ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>;
Array & getDataFromHolderImpl() { return *data; }
const Array & getDataFromHolderImpl() const { return *data; }
public:
/// data_type_ должен быть ненулевым.
ColumnConst(size_t s_, const Array & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>(s_, new Array(data_), data_type_) {}
ColumnConst(size_t s_, const SharedPtr<Array> & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>(s_, data_, data_type_) {}
StringRef getDataAt(size_t n) const override;
StringRef getDataAtWithTerminatingZero(size_t n) const override;
UInt64 get64(size_t n) const override;
/** Более эффективные методы манипуляции */
const Array & getData() const { return *data; }
/** Преобразование из константы в полноценный столбец */
ColumnPtr convertToFullColumn() const override;
void getExtremes(Field & min, Field & max) const override
{
min = FieldType();
max = FieldType();
}
};
@ -158,19 +222,17 @@ template <typename T> ColumnPtr ColumnConst<T>::convertToFullColumn() const
{
ColumnVector<T> * res_ = new ColumnVector<T>;
ColumnPtr res = res_;
res_->getData().assign(s, data);
res_->getData().assign(this->s, this->data);
return res;
}
template <> ColumnPtr ColumnConst<String>::convertToFullColumn() const;
template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const;
template <typename T> StringRef ColumnConst<T>::getDataAt(size_t n) const
{
throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
}
template <> inline StringRef ColumnConst<String>::getDataAt(size_t n) const
@ -180,7 +242,7 @@ template <> inline StringRef ColumnConst<String>::getDataAt(size_t n) const
template <typename T> UInt64 ColumnConst<T>::get64(size_t n) const
{
throw Exception("Method get64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Method get64 is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Для элементарных типов.

View File

@ -38,6 +38,11 @@ public:
{
return amount;
}
Int64 getPeak() const
{
return peak;
}
};

View File

@ -13,6 +13,7 @@
#include <stdint.h>
#include <cstdlib>
#include <stddef.h>
#define ROTL(x,b) (u64)( ((x) << (b)) | ( (x) >> (64 - (b))) )

View File

@ -78,5 +78,7 @@
/// Граница, на которых должны быть выровнены блоки для асинхронных файловых операций.
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096
#define DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR "7500"
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))

View File

@ -31,6 +31,8 @@ ExceptionPtr cloneCurrentException();
void tryLogCurrentException(const char * log_name);
void tryLogCurrentException(Poco::Logger * logger);
std::string getCurrentExceptionMessage(bool with_stacktrace);
void rethrowFirstException(Exceptions & exceptions);

View File

@ -24,6 +24,25 @@ struct BlockIO
Block in_sample; /// Пример блока, который будет прочитан из in.
Block out_sample; /// Пример блока, которого нужно писать в out.
/// Здесь могут быть установлены колбэки для логгирования запроса.
std::function<void(IBlockInputStream &)> finish_callback;
std::function<void()> exception_callback;
/// Вызывайте эти функции, если нужно логгировать запрос.
void onFinish()
{
if (in && finish_callback)
finish_callback(*in);
}
void onException()
{
if (exception_callback)
exception_callback();
else
tryLogCurrentException(__PRETTY_FUNCTION__);
}
BlockIO & operator= (const BlockIO & rhs)
{
/// Обеспечиваем правильный порядок уничтожения.
@ -37,6 +56,9 @@ struct BlockIO
in_sample = rhs.in_sample;
out_sample = rhs.out_sample;
finish_callback = rhs.finish_callback;
exception_callback = rhs.exception_callback;
return *this;
}

View File

@ -136,6 +136,9 @@ protected:
{
createParallelReplicas();
if (settings.skip_unavailable_shards && 0 == parallel_replicas->size())
return Block();
established = true;
parallel_replicas->sendQuery(query, "", stage, true);

View File

@ -15,7 +15,7 @@ namespace DB
class RemoteBlockOutputStream : public IBlockOutputStream
{
public:
RemoteBlockOutputStream(Connection & connection_, const String & query_, Settings * settings_ = nullptr)
RemoteBlockOutputStream(Connection & connection_, const String & query_, const Settings * settings_ = nullptr)
: connection(connection_), query(query_), settings(settings_)
{
}
@ -105,7 +105,7 @@ public:
private:
Connection & connection;
String query;
Settings * settings;
const Settings * settings;
Block sample_block;
};

View File

@ -4,20 +4,25 @@
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
namespace DB
{
/// Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
/** Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
* В случае, если Field - массив, конвертирует все элементы к общему типу.
*/
class FieldToDataType : public StaticVisitor<DataTypePtr>
{
public:
DataTypePtr operator() (const Null & x) const
DataTypePtr operator() (Null & x) const
{
throw Exception("NULL literals is not implemented yet", ErrorCodes::NOT_IMPLEMENTED);
throw Exception("NULL literals are not implemented yet", ErrorCodes::NOT_IMPLEMENTED);
}
DataTypePtr operator() (const UInt64 & x) const
DataTypePtr operator() (UInt64 & x) const
{
if (x <= std::numeric_limits<UInt8>::max()) return new DataTypeUInt8;
if (x <= std::numeric_limits<UInt16>::max()) return new DataTypeUInt16;
@ -25,7 +30,7 @@ public:
return new DataTypeUInt64;
}
DataTypePtr operator() (const Int64 & x) const
DataTypePtr operator() (Int64 & x) const
{
if (x <= std::numeric_limits<Int8>::max() && x >= std::numeric_limits<Int8>::min()) return new DataTypeInt8;
if (x <= std::numeric_limits<Int16>::max() && x >= std::numeric_limits<Int16>::min()) return new DataTypeInt16;
@ -33,22 +38,18 @@ public:
return new DataTypeInt64;
}
DataTypePtr operator() (const Float64 & x) const
DataTypePtr operator() (Float64 & x) const
{
return new DataTypeFloat64;
}
DataTypePtr operator() (const String & x) const
DataTypePtr operator() (String & x) const
{
return new DataTypeString;
}
DataTypePtr operator() (const Array & x) const
{
return new DataTypeArray(apply_visitor(FieldToDataType(), x.at(0)));
}
DataTypePtr operator() (Array & x) const;
};
}

View File

@ -41,7 +41,9 @@ struct ArrayMapImpl
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
{
return new ColumnArray(mapped, array->getOffsetsColumn());
return mapped->isConst()
? new ColumnArray(dynamic_cast<const IColumnConst &>(*mapped).convertToFullColumn(), array->getOffsetsColumn())
: new ColumnArray(mapped, array->getOffsetsColumn());
}
};
@ -59,10 +61,21 @@ struct ArrayFilterImpl
/// Если массивов несколько, сюда передается первый.
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
{
ColumnVector<UInt8> * column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
const ColumnVector<UInt8> * column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
if (!column_filter)
{
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
if (!column_filter_const)
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
if (column_filter_const->getData())
return array->clone();
else
return new ColumnArray(array->getDataPtr()->cloneEmpty(), new ColumnArray::ColumnOffsets_t(array->size(), 0));
}
const IColumn::Filter & filter = column_filter->getData();
ColumnPtr filtered = array->getData().filter(filter);
@ -100,10 +113,35 @@ struct ArrayCountImpl
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
{
ColumnVector<UInt8> * column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
const ColumnVector<UInt8> * column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
if (!column_filter)
{
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
if (!column_filter_const)
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
if (column_filter_const->getData())
{
const IColumn::Offsets_t & offsets = array->getOffsets();
ColumnVector<UInt32> * out_column = new ColumnVector<UInt32>(offsets.size());
ColumnPtr out_column_ptr = out_column;
ColumnVector<UInt32>::Container_t & out_counts = out_column->getData();
size_t pos = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
out_counts[i] = offsets[i] - pos;
pos = offsets[i];
}
return out_column_ptr;
}
else
return new ColumnConstUInt32(array->size(), 0);
}
const IColumn::Filter & filter = column_filter->getData();
const IColumn::Offsets_t & offsets = array->getOffsets();
ColumnVector<UInt32> * out_column = new ColumnVector<UInt32>(offsets.size());
@ -139,10 +177,35 @@ struct ArrayExistsImpl
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
{
ColumnVector<UInt8> * column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
const ColumnVector<UInt8> * column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
if (!column_filter)
{
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
if (!column_filter_const)
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
if (column_filter_const->getData())
{
const IColumn::Offsets_t & offsets = array->getOffsets();
ColumnVector<UInt8> * out_column = new ColumnVector<UInt8>(offsets.size());
ColumnPtr out_column_ptr = out_column;
ColumnVector<UInt8>::Container_t & out_exists = out_column->getData();
size_t pos = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
out_exists[i] = offsets[i] - pos > 0;
pos = offsets[i];
}
return out_column_ptr;
}
else
return new ColumnConstUInt8(array->size(), 0);
}
const IColumn::Filter & filter = column_filter->getData();
const IColumn::Offsets_t & offsets = array->getOffsets();
ColumnVector<UInt8> * out_column = new ColumnVector<UInt8>(offsets.size());
@ -182,10 +245,35 @@ struct ArrayAllImpl
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
{
ColumnVector<UInt8> * column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
const ColumnVector<UInt8> * column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
if (!column_filter)
{
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
if (!column_filter_const)
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
if (column_filter_const->getData())
return new ColumnConstUInt8(array->size(), 1);
else
{
const IColumn::Offsets_t & offsets = array->getOffsets();
ColumnVector<UInt8> * out_column = new ColumnVector<UInt8>(offsets.size());
ColumnPtr out_column_ptr = out_column;
ColumnVector<UInt8>::Container_t & out_all = out_column->getData();
size_t pos = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
out_all[i] = offsets[i] == pos;
pos = offsets[i];
}
return out_column_ptr;
}
}
const IColumn::Filter & filter = column_filter->getData();
const IColumn::Offsets_t & offsets = array->getOffsets();
ColumnVector<UInt8> * out_column = new ColumnVector<UInt8>(offsets.size());
@ -245,8 +333,28 @@ struct ArraySumImpl
const ColumnVector<Element> * column = typeid_cast<const ColumnVector<Element> *>(&*mapped);
if (!column)
{
const ColumnConst<Element> * column_const = typeid_cast<const ColumnConst<Element> *>(&*mapped);
if (!column_const)
return false;
const Element x = column_const->getData();
ColumnVector<Result> * res_column = new ColumnVector<Result>(offsets.size());
res_ptr = res_column;
typename ColumnVector<Result>::Container_t & res = res_column->getData();
size_t pos = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
res[i] = x * (offsets[i] - pos);
pos = offsets[i];
}
return true;
}
const typename ColumnVector<Element>::Container_t & data = column->getData();
ColumnVector<Result> * res_column = new ColumnVector<Result>(offsets.size());
res_ptr = res_column;
@ -300,10 +408,42 @@ struct ArrayFirstImpl
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
{
auto column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
auto column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
if (!column_filter)
{
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
if (!column_filter_const)
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
if (column_filter_const->getData())
{
const auto & offsets = array->getOffsets();
const auto & data = array->getData();
ColumnPtr out{data.cloneEmpty()};
size_t pos{};
for (size_t i = 0; i < offsets.size(); ++i)
{
if (offsets[i] - pos > 0)
out->insert(data[pos]);
else
out->insertDefault();
pos = offsets[i];
}
return out;
}
else
{
ColumnPtr out{array->getData().cloneEmpty()};
out->insertDefault();
return out->replicate(IColumn::Offsets_t(1, array->size()));
}
}
const auto & filter = column_filter->getData();
const auto & offsets = array->getOffsets();
const auto & data = array->getData();
@ -345,10 +485,35 @@ struct ArrayFirstIndexImpl
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
{
auto column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
auto column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
if (!column_filter)
{
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
if (!column_filter_const)
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
if (column_filter_const->getData())
{
const auto & offsets = array->getOffsets();
auto out_column = new ColumnVector<UInt32>{offsets.size()};
ColumnPtr out_column_ptr{out_column};
auto & out_index = out_column->getData();
size_t pos{};
for (size_t i = 0; i < offsets.size(); ++i)
{
out_index[i] = offsets[i] - pos > 0;
pos = offsets[i];
}
return out_column_ptr;
}
else
return new ColumnConstUInt32(array->size(), 0);
}
const auto & filter = column_filter->getData();
const auto & offsets = array->getOffsets();
auto out_column = new ColumnVector<UInt32>{offsets.size()};

View File

@ -89,11 +89,12 @@ namespace DB
static inline ResultType apply(A x)
{
return x < 18 ? 0
return x < 1 ? 0
: (x < 18 ? 17
: (x < 25 ? 18
: (x < 35 ? 25
: (x < 45 ? 35
: 45)));
: 45))));
}
};

View File

@ -447,10 +447,15 @@ struct MatchImpl
const char * str_data = reinterpret_cast<const char *>(&data[i != 0 ? offsets[i - 1] : 0]);
size_t str_size = (i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]) - 1;
/** Даже в случае required_substring_is_prefix используем UNANCHORED проверку регекспа,
* чтобы он мог сматчиться, когда required_substring встречается в строке несколько раз,
* и на первом вхождении регексп не матчит.
*/
if (required_substring_is_prefix)
res[i] = revert ^ regexp->getRE2()->Match(
re2_st::StringPiece(str_data, str_size),
reinterpret_cast<const char *>(pos) - str_data, str_size, re2_st::RE2::ANCHOR_START, nullptr, 0);
reinterpret_cast<const char *>(pos) - str_data, str_size, re2_st::RE2::UNANCHORED, nullptr, 0);
else
res[i] = revert ^ regexp->getRE2()->Match(
re2_st::StringPiece(str_data, str_size),

View File

@ -84,7 +84,7 @@ public:
void ignore(size_t n)
{
while (!eof() && n != 0)
while (n != 0 && !eof())
{
size_t bytes_to_ignore = std::min(static_cast<size_t>(working_buffer.end() - pos), n);
pos += bytes_to_ignore;

View File

@ -105,6 +105,7 @@ void readVectorBinary(std::vector<T> & v, ReadBuffer & buf, size_t MAX_VECTOR_SI
void assertString(const char * s, ReadBuffer & buf);
void assertEOF(ReadBuffer & buf);
void assertChar(char symbol, ReadBuffer & buf);
inline void assertString(const String & s, ReadBuffer & buf)
{

View File

@ -252,13 +252,19 @@ void writeAnyEscapedString(const String & s, WriteBuffer & buf)
}
inline void writeEscapedString(const String & s, WriteBuffer & buf)
inline void writeEscapedString(const char * str, size_t size, WriteBuffer & buf)
{
/// strpbrk в libc под Linux на процессорах с SSE 4.2 хорошо оптимизирована (этот if ускоряет код в 1.5 раза)
if (nullptr == strpbrk(s.data(), "\b\f\n\r\t\'\\") && strlen(s.data()) == s.size())
writeString(s, buf);
if (nullptr == strpbrk(str, "\b\f\n\r\t\'\\") && strlen(str) == size)
writeString(str, size, buf);
else
writeAnyEscapedString<'\''>(s, buf);
writeAnyEscapedString<'\''>(str, str + size, buf);
}
inline void writeEscapedString(const String & s, WriteBuffer & buf)
{
writeEscapedString(s.data(), s.size(), buf);
}
@ -470,6 +476,10 @@ inline void writeText(const Float32 & x, WriteBuffer & buf) { writeFloatText(x,
inline void writeText(const Float64 & x, WriteBuffer & buf) { writeFloatText(x, buf); }
inline void writeText(const String & x, WriteBuffer & buf) { writeEscapedString(x, buf); }
inline void writeText(const bool & x, WriteBuffer & buf) { writeBoolText(x, buf); }
/// в отличие от метода для std::string
/// здесь предполагается, что x null-terminated строка.
inline void writeText(const char * x, WriteBuffer & buf) { writeEscapedString(x, strlen(x), buf); }
inline void writeText(const char * x, size_t size, WriteBuffer & buf) { writeEscapedString(x, size, buf); }
inline void writeText(const VisitID_t & x, WriteBuffer & buf) { writeIntText(static_cast<const UInt64 &>(x), buf); }
inline void writeText(const mysqlxx::Date & x, WriteBuffer & buf) { writeDateText(x, buf); }

View File

@ -40,6 +40,7 @@ class ProcessListElement;
class Macros;
class Progress;
class Clusters;
class QueryLog;
/// имя таблицы -> таблица
@ -64,12 +65,30 @@ typedef std::vector<DatabaseAndTableName> Dependencies;
*/
class Context
{
public:
enum class Interface
{
TCP = 1,
HTTP = 2,
OLAP_HTTP = 3,
};
enum class HTTPMethod
{
UNKNOWN = 0,
GET = 1,
POST = 2,
};
private:
typedef std::shared_ptr<ContextShared> Shared;
Shared shared;
String user; /// Текущий пользователь.
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
Interface interface = Interface::TCP;
HTTPMethod http_method = HTTPMethod::UNKNOWN; /// NOTE Возможно, перенести это в отдельный struct ClientInfo.
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
String current_database; /// Текущая БД.
String current_query_id; /// Id текущего запроса.
@ -108,6 +127,12 @@ public:
String getUser() const { return user; }
Poco::Net::IPAddress getIPAddress() const { return ip_address; }
Interface getInterface() const { return interface; }
void setInterface(Interface interface_) { interface = interface_; }
HTTPMethod getHTTPMethod() const { return http_method; }
void setHTTPMethod(HTTPMethod http_method_) { http_method = http_method_; }
void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address);
QuotaForIntervals & getQuota();
@ -242,6 +267,8 @@ public:
Compiler & getCompiler();
QueryLog & getQueryLog();
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;

View File

@ -5,8 +5,8 @@
#include <Poco/Net/IPAddress.h>
#include <DB/Core/Types.h>
#include <DB/Common/ConcurrentBoundedQueue.h>
#include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h>
#include <statdaemons/Stopwatch.h>
namespace DB
@ -34,40 +34,33 @@ struct QueryLogElement
SHUTDOWN = 0, /// Эта запись имеет служебное значение.
QUERY_START = 1,
QUERY_FINISH = 2,
EXCEPTION_BEFORE_START = 3,
EXCEPTION_WHILE_PROCESSING = 4,
};
enum Interface
{
TCP = 1,
HTTP = 2,
OLAP_HTTP = 3,
};
enum HTTPMethod
{
UNKNOWN = 0,
GET = 1,
POST = 2,
};
Type type;
Type type = QUERY_START;
/// В зависимости от типа, не все поля могут быть заполнены.
time_t event_time;
time_t query_start_time;
UInt64 query_duration_ms;
time_t event_time{};
time_t query_start_time{};
UInt64 query_duration_ms{};
UInt64 read_rows;
UInt64 read_bytes;
UInt64 read_rows{};
UInt64 read_bytes{};
UInt64 result_rows;
UInt64 result_bytes;
UInt64 result_rows{};
UInt64 result_bytes{};
UInt64 memory_usage{};
String query;
Interface interface;
HTTPMethod http_method;
String exception;
String stack_trace;
Context::Interface interface = Context::Interface::TCP;
Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN;
Poco::Net::IPAddress ip_address;
String user;
String query_id;
@ -76,6 +69,8 @@ struct QueryLogElement
#define DBMS_QUERY_LOG_QUEUE_SIZE 1024
class Context;
class QueryLog : private boost::noncopyable
{
@ -89,25 +84,8 @@ public:
* где N - минимальное число, начиная с 1 такое, что таблицы с таким именем ещё нет;
* и создаётся новая таблица, как будто существующей таблицы не было.
*/
QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_)
: context(context_), database_name(database_name_), table_name(table_name_), flush_interval_milliseconds(flush_interval_milliseconds_)
{
data.reserve(DBMS_QUERY_LOG_QUEUE_SIZE);
// TODO
saving_thread = std::thread([this] { threadFunction(); });
}
~QueryLog()
{
/// Говорим потоку, что надо завершиться.
QueryLogElement elem;
elem.type = QueryLogElement::SHUTDOWN;
queue.push(elem);
saving_thread.join();
}
QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_);
~QueryLog();
/** Добавить запись в лог.
* Сохранение в таблицу делается асинхронно, и в случае сбоя, запись может никуда не попасть.
@ -134,111 +112,15 @@ private:
*/
std::vector<QueryLogElement> data;
Logger * log {&Logger::get("QueryLog")};
/** В этом потоке данные вынимаются из queue, складываются в data, а затем вставляются в таблицу.
*/
std::thread saving_thread;
void threadFunction()
{
Stopwatch time_after_last_write;
bool first = true;
while (true)
{
try
{
if (first)
{
time_after_last_write.restart();
first = false;
}
QueryLogElement element;
bool has_element = false;
if (data.empty())
{
element = queue.pop();
has_element = true;
}
else
{
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
if (milliseconds_elapsed < flush_interval_milliseconds)
has_element = queue.tryPop(element, flush_interval_milliseconds - milliseconds_elapsed);
}
if (has_element)
{
if (element.type = QueryLogElement::SHUTDOWN)
{
flush();
break;
}
else
data.push_back(element);
}
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
if (milliseconds_elapsed >= flush_interval_milliseconds)
{
/// Записываем данные в таблицу.
flush();
time_after_last_write.restart();
}
}
catch (...)
{
/// В случае ошибки теряем накопленные записи, чтобы не блокироваться.
data.clear();
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
Block createBlock()
{
return {
{new ColumnUInt8, new DataTypeUInt8, "type"},
{new ColumnUInt32, new DataTypeDateTime, "event_time"},
{new ColumnUInt32, new DataTypeDateTime, "query_start_time"},
};
/* time_t event_time;
time_t query_start_time;
UInt64 query_duration_ms;
UInt64 read_rows;
UInt64 read_bytes;
UInt64 result_rows;
UInt64 result_bytes;
String query;
Interface interface;
HTTPMethod http_method;
Poco::Net::IPAddress ip_address;
String user;
String query_id;*/
}
void flush()
{
try
{
Block block = createBlock();
// TODO Формирование блока и запись.
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
data.clear();
}
void threadFunction();
static Block createBlock();
void flush();
};

View File

@ -83,7 +83,7 @@ struct QuotaForInterval
/// Увеличить соответствующее значение.
void addQuery(time_t current_time, const String & quota_name);
void addError(time_t current_time, const String & quota_name);
void addError(time_t current_time, const String & quota_name) noexcept;
/// Проверить, не превышена ли квота уже. Если превышена - кидает исключение.
void checkExceeded(time_t current_time, const String & quota_name);
@ -139,7 +139,7 @@ public:
void setMax(const QuotaForIntervals & quota);
void addQuery(time_t current_time);
void addError(time_t current_time);
void addError(time_t current_time) noexcept;
void checkExceeded(time_t current_time);

View File

@ -97,6 +97,14 @@ struct Settings
M(SettingUInt64, parallel_replicas_count, 0) \
M(SettingUInt64, parallel_replica_offset, 0) \
\
/** Тихо пропускать недоступные шарды. */ \
M(SettingBool, skip_unavailable_shards, false) \
\
/** Не мерджить состояния агрегации с разных серверов при распределённой обработке запроса \
* - на случай, когда доподлинно известно, что на разных шардах разные ключи. \
*/ \
M(SettingBool, distributed_group_by_no_merge, false) \
\
/** Тонкие настройки для чтения из MergeTree */ \
\
/** Если из одного файла читается хотя бы столько строк, чтение можно распараллелить. */ \
@ -135,6 +143,9 @@ struct Settings
\
/** Приоритет запроса. 1 - самый высокий, больше - ниже; 0 - не использовать приоритеты. */ \
M(SettingUInt64, priority, 0) \
\
/** Логгировать запросы и писать лог в системную таблицу. */ \
M(SettingBool, log_queries, 0) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

@ -20,7 +20,7 @@ public:
ASTLiteral() = default;
ASTLiteral(const StringRange range_, const Field & value_) : ASTWithAlias(range_), value(value_) {}
String getColumnName() const override { return apply_visitor(FieldVisitorToString(), value); }
String getColumnName() const override;
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "Literal_" + apply_visitor(FieldVisitorDump(), value); }

View File

@ -6,7 +6,6 @@
namespace DB
{
/** Запрос с секцией FORMAT.
*/
class ASTQueryWithOutput : public IAST
@ -16,6 +15,11 @@ public:
ASTQueryWithOutput() = default;
ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
/** Возвращает указатель на формат. Если типом объекта является ASTSelectQuery,
* то эта функция возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
*/
virtual const IAST * getFormat() const { return format.get(); }
};

View File

@ -34,6 +34,9 @@ public:
ASTPtr clone() const override;
/// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
const IAST * getFormat() const override;
public:
bool distinct = false;
ASTPtr select_expression_list;

View File

@ -100,6 +100,20 @@ protected:
};
/** Массив литералов.
* Массивы могут распарситься и как применение оператора [].
* Но парсинг всего массива как целой константы серьёзно ускоряет анализ выражений в случае очень больших массивов.
* Мы пробуем распарсить массив как массив литералов сначала (fast path),
* а если не получилось (когда массив состоит из сложных выражений) - парсим как применение оператора [] (slow path).
*/
class ParserArrayOfLiterals : public IParserBase
{
protected:
const char * getName() const { return "array"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
/** Литерал - одно из: NULL, UInt64, Int64, Float64, String.
*/
class ParserLiteral : public IParserBase

View File

@ -15,115 +15,13 @@ namespace DB
{
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
/** Более точное сравнение.
* Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой.
* Правила сравнения - такие же, что и в FunctionsComparison.
* В том числе, сравнение знаковых и беззнаковых оставляем UB.
*/
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
{
public:
bool operator() (const Null & l, const Null & r) const { return true; }
bool operator() (const Null & l, const UInt64 & r) const { return false; }
bool operator() (const Null & l, const Int64 & r) const { return false; }
bool operator() (const Null & l, const Float64 & r) const { return false; }
bool operator() (const Null & l, const String & r) const { return false; }
bool operator() (const Null & l, const Array & r) const { return false; }
bool operator() (const UInt64 & l, const Null & r) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const String & r) const { return false; }
bool operator() (const UInt64 & l, const Array & r) const { return false; }
bool operator() (const Int64 & l, const Null & r) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
bool operator() (const Int64 & l, const Float64 & r) const { return l == r; }
bool operator() (const Int64 & l, const String & r) const { return false; }
bool operator() (const Int64 & l, const Array & r) const { return false; }
bool operator() (const Float64 & l, const Null & r) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const Float64 & l, const Int64 & r) const { return l == r; }
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
bool operator() (const Float64 & l, const String & r) const { return false; }
bool operator() (const Float64 & l, const Array & r) const { return false; }
bool operator() (const String & l, const Null & r) const { return false; }
bool operator() (const String & l, const UInt64 & r) const { return false; }
bool operator() (const String & l, const Int64 & r) const { return false; }
bool operator() (const String & l, const Float64 & r) const { return false; }
bool operator() (const String & l, const String & r) const { return l == r; }
bool operator() (const String & l, const Array & r) const { return false; }
bool operator() (const Array & l, const Null & r) const { return false; }
bool operator() (const Array & l, const UInt64 & r) const { return false; }
bool operator() (const Array & l, const Int64 & r) const { return false; }
bool operator() (const Array & l, const Float64 & r) const { return false; }
bool operator() (const Array & l, const String & r) const { return false; }
bool operator() (const Array & l, const Array & r) const { return l == r; }
};
class FieldVisitorAccurateLess : public StaticVisitor<bool>
{
public:
bool operator() (const Null & l, const Null & r) const { return false; }
bool operator() (const Null & l, const UInt64 & r) const { return true; }
bool operator() (const Null & l, const Int64 & r) const { return true; }
bool operator() (const Null & l, const Float64 & r) const { return true; }
bool operator() (const Null & l, const String & r) const { return true; }
bool operator() (const Null & l, const Array & r) const { return true; }
bool operator() (const UInt64 & l, const Null & r) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const String & r) const { return true; }
bool operator() (const UInt64 & l, const Array & r) const { return true; }
bool operator() (const Int64 & l, const Null & r) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
bool operator() (const Int64 & l, const Float64 & r) const { return l < r; }
bool operator() (const Int64 & l, const String & r) const { return true; }
bool operator() (const Int64 & l, const Array & r) const { return true; }
bool operator() (const Float64 & l, const Null & r) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const Float64 & l, const Int64 & r) const { return l < r; }
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
bool operator() (const Float64 & l, const String & r) const { return true; }
bool operator() (const Float64 & l, const Array & r) const { return true; }
bool operator() (const String & l, const Null & r) const { return false; }
bool operator() (const String & l, const UInt64 & r) const { return false; }
bool operator() (const String & l, const Int64 & r) const { return false; }
bool operator() (const String & l, const Float64 & r) const { return false; }
bool operator() (const String & l, const String & r) const { return l < r; }
bool operator() (const String & l, const Array & r) const { return true; }
bool operator() (const Array & l, const Null & r) const { return false; }
bool operator() (const Array & l, const UInt64 & r) const { return false; }
bool operator() (const Array & l, const Int64 & r) const { return false; }
bool operator() (const Array & l, const Float64 & r) const { return false; }
bool operator() (const Array & l, const String & r) const { return false; }
bool operator() (const Array & l, const Array & r) const { return l < r; }
};
#pragma GCC diagnostic pop
/** Диапазон с открытыми или закрытыми концами; возможно, неограниченный.
*/
struct Range
{
private:
static bool equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
static bool less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
static bool equals(const Field & lhs, const Field & rhs);
static bool less(const Field & lhs, const Field & rhs);
public:
Field left; /// левая граница, если есть

View File

@ -96,6 +96,7 @@ private:
};
bool is_interactive = true; /// Использовать readline интерфейс или batch режим.
bool need_render_progress = true; /// Рисовать прогресс выполнения запроса.
bool print_time_to_stderr = false; /// В неинтерактивном режиме, выводить время выполнения в stderr.
bool stdin_is_not_tty = false; /// stdin - не терминал.
@ -268,6 +269,9 @@ private:
insert_format = "Values";
insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size);
if (!is_interactive)
need_render_progress = config().getBool("progress", false);
connect();
if (is_interactive)
@ -850,11 +854,11 @@ private:
/// Формат может быть указан в запросе.
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
{
if (query_with_output->format)
if (query_with_output->getFormat() != nullptr)
{
if (has_vertical_output_suffix)
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(&*query_with_output->format))
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(query_with_output->getFormat()))
current_format = id->name;
}
}
@ -906,7 +910,7 @@ private:
void writeProgress()
{
if (!is_interactive)
if (!need_render_progress)
return;
static size_t increment = 0;
@ -1053,6 +1057,7 @@ public:
("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command")
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
("stacktrace", "print stack traces of exceptions")
("progress", "print progress even in non-interactive mode")
APPLY_FOR_SETTINGS(DECLARE_SETTING)
APPLY_FOR_LIMITS(DECLARE_LIMIT)
;
@ -1171,6 +1176,8 @@ public:
config().setBool("vertical", true);
if (options.count("stacktrace"))
config().setBool("stacktrace", true);
if (options.count("progress"))
config().setBool("progress", true);
if (options.count("time"))
print_time_to_stderr = true;
}

View File

@ -3,7 +3,7 @@
namespace DB
{
ParallelReplicas::ParallelReplicas(Connection * connection_, Settings * settings_, ThrottlerPtr throttler_)
ParallelReplicas::ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_)
: settings(settings_), throttler(throttler_),
active_replica_count(1),
supports_parallel_execution(false)
@ -11,7 +11,7 @@ ParallelReplicas::ParallelReplicas(Connection * connection_, Settings * settings
registerReplica(connection_);
}
ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_, ThrottlerPtr throttler_)
ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_)
: settings(settings_), throttler(throttler_)
{
if (pool_ == nullptr)
@ -37,6 +37,7 @@ ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_
supports_parallel_execution = false;
pool_entry = pool_->get(settings);
if (!pool_entry.isNull())
registerReplica(&*pool_entry);
}
}

View File

@ -1,5 +1,6 @@
#include <DB/DataTypes/FieldToDataType.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnConst.h>
@ -61,7 +62,7 @@ template <> ColumnPtr ColumnConst<String>::convertToFullColumn() const
}
template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
ColumnPtr ColumnConst<Array>::convertToFullColumn() const
{
if (!data_type)
throw Exception("No data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR);
@ -70,7 +71,8 @@ template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
if (!type)
throw Exception("Non-array data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR);
size_t array_size = data.size();
const Array & array = getDataFromHolderImpl();
size_t array_size = array.size();
ColumnPtr nested_column = type->getNestedType()->createColumn();
ColumnArray * res = new ColumnArray(nested_column);
@ -81,11 +83,27 @@ template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
{
offsets[i] = (i + 1) * array_size;
for (size_t j = 0; j < array_size; ++j)
nested_column->insert(data[j]);
nested_column->insert(array[j]);
}
return res;
}
StringRef ColumnConst<Array>::getDataAt(size_t n) const
{
throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
}
UInt64 ColumnConst<Array>::get64(size_t n) const
{
throw Exception("Method get64 is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
}
StringRef ColumnConst<Array>::getDataAtWithTerminatingZero(size_t n) const
{
throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
}
}

View File

@ -59,6 +59,19 @@ void tryLogCurrentException(const char * log_name)
void tryLogCurrentException(Poco::Logger * logger)
{
try
{
LOG_ERROR(logger, getCurrentExceptionMessage(true));
}
catch (...)
{
}
}
std::string getCurrentExceptionMessage(bool with_stacktrace)
{
std::stringstream stream;
try
{
throw;
@ -67,8 +80,10 @@ void tryLogCurrentException(Poco::Logger * logger)
{
try
{
LOG_ERROR(logger, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
stream << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
if (with_stacktrace)
stream << ", Stack trace:\n\n" << e.getStackTrace().toString();
}
catch (...) {}
}
@ -76,8 +91,8 @@ void tryLogCurrentException(Poco::Logger * logger)
{
try
{
LOG_ERROR(logger, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
}
catch (...) {}
}
@ -91,7 +106,7 @@ void tryLogCurrentException(Poco::Logger * logger)
if (status)
name += " (demangling status: " + toString(status) + ")";
LOG_ERROR(logger, "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what());
stream << "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what();
}
catch (...) {}
}
@ -105,10 +120,12 @@ void tryLogCurrentException(Poco::Logger * logger)
if (status)
name += " (demangling status: " + toString(status) + ")";
LOG_ERROR(logger, "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name);
stream << "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name;
}
catch (...) {}
}
return stream.str();
}

View File

@ -0,0 +1,170 @@
#include <DB/DataTypes/FieldToDataType.h>
namespace DB
{
template <typename T>
static void convertArrayToCommonType(Array & arr)
{
for (auto & elem : arr)
elem = apply_visitor(FieldVisitorConvertToNumber<T>(), elem);
}
DataTypePtr FieldToDataType::operator() (Array & x) const
{
if (x.empty())
throw Exception("Cannot infer type of empty array", ErrorCodes::EMPTY_DATA_PASSED);
/** Тип массива нужно вывести по типу его элементов.
* Если элементы - числа, то нужно выбрать наименьший общий тип, если такой есть,
* или кинуть исключение.
* Код похож на NumberTraits::ResultOfIf, но тем кодом трудно здесь непосредственно воспользоваться.
*
* Также заметим, что Float32 не выводится, вместо этого используется только Float64.
* Это сделано потому что литералов типа Float32 не бывает в запросе.
*/
bool has_string = false;
bool has_array = false;
bool has_float = false;
int max_bits = 0;
int max_signed_bits = 0;
int max_unsigned_bits = 0;
for (const Field & elem : x)
{
switch (elem.getType())
{
case Field::Types::UInt64:
{
UInt64 num = elem.get<UInt64>();
if (num <= std::numeric_limits<UInt8>::max())
max_unsigned_bits = std::max(8, max_unsigned_bits);
else if (num <= std::numeric_limits<UInt16>::max())
max_unsigned_bits = std::max(16, max_unsigned_bits);
else if (num <= std::numeric_limits<UInt32>::max())
max_unsigned_bits = std::max(32, max_unsigned_bits);
else
max_unsigned_bits = 64;
max_bits = std::max(max_unsigned_bits, max_bits);
break;
}
case Field::Types::Int64:
{
Int64 num = elem.get<Int64>();
if (num <= std::numeric_limits<Int8>::max() && num >= std::numeric_limits<Int8>::min())
max_signed_bits = std::max(8, max_signed_bits);
else if (num <= std::numeric_limits<Int16>::max() && num >= std::numeric_limits<Int16>::min())
max_signed_bits = std::max(16, max_signed_bits);
else if (num <= std::numeric_limits<Int32>::max() && num >= std::numeric_limits<Int32>::min())
max_signed_bits = std::max(32, max_signed_bits);
else
max_signed_bits = 64;
max_bits = std::max(max_signed_bits, max_bits);
break;
}
case Field::Types::Float64:
{
has_float = true;
break;
}
case Field::Types::String:
{
has_string = true;
break;
}
case Field::Types::Array:
{
has_array = true;
break;
}
case Field::Types::Null:
{
throw Exception("NULL literals are not implemented yet", ErrorCodes::NOT_IMPLEMENTED);
break;
}
}
}
if ((has_string + has_array + (max_bits > 0)) > 1)
throw Exception("Incompatible types of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (has_array)
throw Exception("Type inference of multidimensional arrays is not supported", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (has_string)
return new DataTypeArray(new DataTypeString);
if (has_float && max_bits == 64)
throw Exception("Incompatible types Float64 and UInt64/Int64 of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (has_float)
{
convertArrayToCommonType<Float64>(x);
return new DataTypeArray(new DataTypeFloat64);
}
if (max_signed_bits == 64 && max_unsigned_bits == 64)
throw Exception("Incompatible types UInt64 and Int64 of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (max_signed_bits && !max_unsigned_bits)
{
if (max_signed_bits == 8)
return new DataTypeArray(new DataTypeInt8);
if (max_signed_bits == 16)
return new DataTypeArray(new DataTypeInt16);
if (max_signed_bits == 32)
return new DataTypeArray(new DataTypeInt32);
if (max_signed_bits == 64)
return new DataTypeArray(new DataTypeInt64);
}
if (!max_signed_bits && max_unsigned_bits)
{
if (max_unsigned_bits == 8)
return new DataTypeArray(new DataTypeUInt8);
if (max_unsigned_bits == 16)
return new DataTypeArray(new DataTypeUInt16);
if (max_unsigned_bits == 32)
return new DataTypeArray(new DataTypeUInt32);
if (max_unsigned_bits == 64)
return new DataTypeArray(new DataTypeUInt64);
}
if (max_signed_bits && max_unsigned_bits)
{
convertArrayToCommonType<Int64>(x);
if (max_unsigned_bits >= max_signed_bits)
{
/// Беззнаковый тип не помещается в знаковый. Надо увеличить количество бит.
if (max_bits == 8)
return new DataTypeArray(new DataTypeInt16);
if (max_bits == 16)
return new DataTypeArray(new DataTypeInt32);
if (max_bits == 32)
return new DataTypeArray(new DataTypeInt64);
else
throw Exception("Incompatible types UInt64 and signed integer of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
else
{
/// Беззнаковый тип помещается в знаковый.
if (max_bits == 8)
return new DataTypeArray(new DataTypeInt8);
if (max_bits == 16)
return new DataTypeArray(new DataTypeInt16);
if (max_bits == 32)
return new DataTypeArray(new DataTypeInt32);
if (max_bits == 64)
return new DataTypeArray(new DataTypeInt64);
}
}
throw Exception("Incompatible types of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}

View File

@ -36,6 +36,16 @@ void assertString(const char * s, ReadBuffer & buf)
}
}
void assertChar(char symbol, ReadBuffer & buf)
{
if (buf.eof() || *buf.position() != symbol)
{
char err[2] = {symbol, '\0'};
throwAtAssertionFailed(err, buf);
}
++buf.position();
}
void assertEOF(ReadBuffer & buf)
{
if (!buf.eof())

View File

@ -557,14 +557,21 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
for (size_t i = 0; i < aggregates_size; ++i)
aggregate_columns[i].resize(aggregates[i].arguments.size());
/** Константные столбцы не поддерживаются напрямую при агрегации.
* Чтобы они всё-равно работали, материализуем их.
*/
Columns materialized_columns;
/// Запоминаем столбцы, с которыми будем работать
for (size_t i = 0; i < keys_size; ++i)
{
key_columns[i] = block.getByPosition(keys[i]).column;
if (key_columns[i]->isConst())
throw Exception("Constants are not allowed as GROUP BY keys"
" (but all of them must be eliminated in ExpressionAnalyzer)", ErrorCodes::ILLEGAL_COLUMN);
if (const IColumnConst * column_const = dynamic_cast<const IColumnConst *>(key_columns[i]))
{
materialized_columns.push_back(column_const->convertToFullColumn());
key_columns[i] = materialized_columns.back().get();
}
}
for (size_t i = 0; i < aggregates_size; ++i)
@ -573,11 +580,11 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
{
aggregate_columns[i][j] = block.getByPosition(aggregates[i].arguments[j]).column;
/** Агрегатные функции рассчитывают, что в них передаются полноценные столбцы.
* Поэтому, стобцы-константы не разрешены в качестве аргументов агрегатных функций.
*/
if (aggregate_columns[i][j]->isConst())
throw Exception("Constants are not allowed as arguments of aggregate functions", ErrorCodes::ILLEGAL_COLUMN);
if (const IColumnConst * column_const = dynamic_cast<const IColumnConst *>(aggregate_columns[i][j]))
{
materialized_columns.push_back(column_const->convertToFullColumn());
aggregate_columns[i][j] = materialized_columns.back().get();
}
}
}

View File

@ -27,6 +27,7 @@
#include <DB/Interpreters/Cluster.h>
#include <DB/Interpreters/InterserverIOHandler.h>
#include <DB/Interpreters/Compiler.h>
#include <DB/Interpreters/QueryLog.h>
#include <DB/Interpreters/Context.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/WriteBufferFromString.h>
@ -86,6 +87,7 @@ struct ContextShared
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
Macros macros; /// Подстановки из конфига.
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
std::unique_ptr<QueryLog> query_log; /// Для логгирования запросов.
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска.
/// Кластеры для distributed таблиц
@ -830,6 +832,26 @@ Compiler & Context::getCompiler()
}
QueryLog & Context::getQueryLog()
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (!shared->query_log)
{
auto & config = Poco::Util::Application::instance().config();
String database = config.getString("query_log.database", "system");
String table = config.getString("query_log.table", "query_log");
size_t flush_interval_milliseconds = parse<size_t>(
config.getString("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR));
shared->query_log.reset(new QueryLog{ *this, database, table, flush_interval_milliseconds });
}
return *shared->query_log;
}
CompressionMethod Context::chooseCompressionMethod(size_t part_size, double part_size_ratio) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);

View File

@ -1399,6 +1399,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(&*ast))
{
DataTypePtr type = apply_visitor(FieldToDataType(), node->value);
ColumnWithNameAndType column;
column.column = type->createConstColumn(1, node->value);
column.type = type;

View File

@ -81,6 +81,10 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
{
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
const auto & lifetime = dict_ptr->getLifetime();
std::uniform_int_distribution<std::uint64_t> distribution{lifetime.min_sec, lifetime.max_sec};
update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
const auto dict_it = dictionaries.find(name);
if (dict_it->second.dict)
dict_it->second.dict->set(dict_ptr.release());
@ -90,10 +94,6 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
/// erase stored exception on success
dict_it->second.exception = std::exception_ptr{};
const auto & lifetime = dict_ptr->getLifetime();
std::uniform_int_distribution<std::uint64_t> distribution{lifetime.min_sec, lifetime.max_sec};
update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
recreated_failed_dictionaries.push_back(name);
}
}

View File

@ -0,0 +1,279 @@
#include <statdaemons/Stopwatch.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Parsers/ExpressionElementParsers.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/Interpreters/InterpreterCreateQuery.h>
#include <DB/Interpreters/InterpreterRenameQuery.h>
#include <DB/Interpreters/QueryLog.h>
namespace DB
{
QueryLog::QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_)
: context(context_), database_name(database_name_), table_name(table_name_), flush_interval_milliseconds(flush_interval_milliseconds_)
{
data.reserve(DBMS_QUERY_LOG_QUEUE_SIZE);
{
String description = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name);
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
table = context.tryGetTable(database_name, table_name);
if (table)
{
const Block expected = createBlock();
const Block actual = table->getSampleBlockNonMaterialized();
if (!blocksHaveEqualStructure(actual, expected))
{
/// Переименовываем существующую таблицу.
int suffix = 0;
while (context.isTableExist(database_name, table_name + "_" + toString(suffix)))
++suffix;
ASTRenameQuery * rename = new ASTRenameQuery;
ASTPtr holder = rename;
ASTRenameQuery::Table from;
from.database = database_name;
from.table = table_name;
ASTRenameQuery::Table to;
to.database = database_name;
to.table = table_name + "_" + toString(suffix);
ASTRenameQuery::Element elem;
elem.from = from;
elem.to = to;
rename->elements.emplace_back(elem);
LOG_DEBUG(log, "Existing table " << description << " for query log has obsolete or different structure."
" Renaming it to " << backQuoteIfNeed(to.table));
InterpreterRenameQuery(holder, context).execute();
/// Нужная таблица будет создана.
table = nullptr;
}
else
LOG_DEBUG(log, "Will use existing table " << description << " for query log.");
}
if (!table)
{
/// Создаём таблицу.
LOG_DEBUG(log, "Creating new table " << description << " for query log.");
ASTCreateQuery * create = new ASTCreateQuery;
ASTPtr holder = create;
create->database = database_name;
create->table = table_name;
Block sample = createBlock();
create->columns = InterpreterCreateQuery::formatColumns(sample.getColumnsList());
String engine = "MergeTree(event_date, event_time, 8192)";
ParserFunction engine_parser;
create->storage = parseQuery(engine_parser, engine.data(), engine.data() + engine.size(), "ENGINE to create table for query log");
InterpreterCreateQuery(holder, context).execute();
table = context.getTable(database_name, table_name);
}
}
saving_thread = std::thread([this] { threadFunction(); });
}
QueryLog::~QueryLog()
{
/// Говорим потоку, что надо завершиться.
QueryLogElement elem;
elem.type = QueryLogElement::SHUTDOWN;
queue.push(elem);
saving_thread.join();
}
void QueryLog::threadFunction()
{
Stopwatch time_after_last_write;
bool first = true;
while (true)
{
try
{
if (first)
{
time_after_last_write.restart();
first = false;
}
QueryLogElement element;
bool has_element = false;
if (data.empty())
{
queue.pop(element);
has_element = true;
}
else
{
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
if (milliseconds_elapsed < flush_interval_milliseconds)
has_element = queue.tryPop(element, flush_interval_milliseconds - milliseconds_elapsed);
}
if (has_element)
{
if (element.type == QueryLogElement::SHUTDOWN)
{
flush();
break;
}
else
data.push_back(element);
}
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
if (milliseconds_elapsed >= flush_interval_milliseconds)
{
/// Записываем данные в таблицу.
flush();
time_after_last_write.restart();
}
}
catch (...)
{
/// В случае ошибки теряем накопленные записи, чтобы не блокироваться.
data.clear();
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
Block QueryLog::createBlock()
{
return
{
{new ColumnUInt8, new DataTypeUInt8, "type"},
{new ColumnUInt16, new DataTypeDate, "event_date"},
{new ColumnUInt32, new DataTypeDateTime, "event_time"},
{new ColumnUInt32, new DataTypeDateTime, "query_start_time"},
{new ColumnUInt64, new DataTypeUInt64, "query_duration_ms"},
{new ColumnUInt64, new DataTypeUInt64, "read_rows"},
{new ColumnUInt64, new DataTypeUInt64, "read_bytes"},
{new ColumnUInt64, new DataTypeUInt64, "result_rows"},
{new ColumnUInt64, new DataTypeUInt64, "result_bytes"},
{new ColumnUInt64, new DataTypeUInt64, "memory_usage"},
{new ColumnString, new DataTypeString, "query"},
{new ColumnString, new DataTypeString, "exception"},
{new ColumnString, new DataTypeString, "stack_trace"},
{new ColumnUInt8, new DataTypeUInt8, "interface"},
{new ColumnUInt8, new DataTypeUInt8, "http_method"},
{new ColumnFixedString(16), new DataTypeFixedString(16), "ip_address"},
{new ColumnString, new DataTypeString, "user"},
{new ColumnString, new DataTypeString, "query_id"},
};
}
void QueryLog::flush()
{
try
{
LOG_TRACE(log, "Flushing query log");
DateLUT & date_lut = DateLUT::instance();
Block block = createBlock();
for (const QueryLogElement & elem : data)
{
char ipv6_binary[16];
if (Poco::Net::IPAddress::IPv6 == elem.ip_address.family())
{
memcpy(ipv6_binary, elem.ip_address.addr(), 16);
}
else if (Poco::Net::IPAddress::IPv4 == elem.ip_address.family())
{
/// Преобразуем в IPv6-mapped адрес.
memset(ipv6_binary, 0, 10);
ipv6_binary[10] = '\xFF';
ipv6_binary[11] = '\xFF';
memcpy(&ipv6_binary[12], elem.ip_address.addr(), 4);
}
else
memset(ipv6_binary, 0, 16);
size_t i = 0;
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.type));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(date_lut.toDayNum(elem.event_time)));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.event_time));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.query_start_time));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.query_duration_ms));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.read_rows));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.read_bytes));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.result_rows));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.result_bytes));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.memory_usage));
block.unsafeGetByPosition(i++).column.get()->insertData(elem.query.data(), elem.query.size());
block.unsafeGetByPosition(i++).column.get()->insertData(elem.exception.data(), elem.exception.size());
block.unsafeGetByPosition(i++).column.get()->insertData(elem.stack_trace.data(), elem.stack_trace.size());
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.interface));
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.http_method));
block.unsafeGetByPosition(i++).column.get()->insertData(ipv6_binary, 16);
block.unsafeGetByPosition(i++).column.get()->insertData(elem.user.data(), elem.user.size());
block.unsafeGetByPosition(i++).column.get()->insertData(elem.query_id.data(), elem.query_id.size());
}
BlockOutputStreamPtr stream = table->write(nullptr);
stream->writePrefix();
stream->write(block);
stream->writeSuffix();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
/// В случае ошибки тоже очищаем накопленные записи, чтобы не блокироваться.
data.clear();
}
}

View File

@ -66,7 +66,7 @@ void QuotaForInterval::addQuery(time_t current_time, const String & quota_name)
__sync_fetch_and_add(&used.queries, 1);
}
void QuotaForInterval::addError(time_t current_time, const String & quota_name)
void QuotaForInterval::addError(time_t current_time, const String & quota_name) noexcept
{
__sync_fetch_and_add(&used.errors, 1);
}
@ -181,7 +181,7 @@ void QuotaForIntervals::addQuery(time_t current_time)
it->second.addQuery(current_time, name);
}
void QuotaForIntervals::addError(time_t current_time)
void QuotaForIntervals::addError(time_t current_time) noexcept
{
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
it->second.addError(current_time, name);

View File

@ -1,10 +1,12 @@
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/formatReadable.h>
#include <DB/IO/ConcatReadBuffer.h>
#include <DB/DataStreams/BlockIO.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/DataStreams/copyData.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
@ -15,6 +17,7 @@
#include <DB/Interpreters/Quota.h>
#include <DB/Interpreters/InterpreterFactory.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Interpreters/QueryLog.h>
#include <DB/Interpreters/executeQuery.h>
@ -31,6 +34,7 @@ static void checkLimits(const IAST & ast, const Limits & limits)
}
/// Логгировать запрос в обычный лог (не в таблицу).
static void logQuery(const String & query, const Context & context)
{
String logged_query = query;
@ -39,6 +43,53 @@ static void logQuery(const String & query, const Context & context)
}
static void setClientInfo(QueryLogElement & elem, Context & context)
{
elem.interface = context.getInterface();
elem.http_method = context.getHTTPMethod();
elem.ip_address = context.getIPAddress();
elem.user = context.getUser();
elem.query_id = context.getCurrentQueryId();
}
static void onExceptionBeforeStart(const String & query, Context & context, time_t current_time)
{
/// Эксепшен до начала выполнения запроса.
context.getQuota().addError(current_time);
bool log_queries = context.getSettingsRef().log_queries;
/// Логгируем в таблицу начало выполнения запроса, если нужно.
if (log_queries)
{
QueryLogElement elem;
elem.type = QueryLogElement::EXCEPTION_BEFORE_START;
elem.event_time = current_time;
elem.query_start_time = current_time;
elem.query = query;
elem.exception = getCurrentExceptionMessage(false);
setClientInfo(elem, context);
try
{
throw;
}
catch (const Exception & e)
{
elem.stack_trace = e.getStackTrace().toString();
}
catch (...) {}
context.getQueryLog().add(elem);
}
}
static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
IParser::Pos begin,
IParser::Pos end,
@ -47,6 +98,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
QueryProcessingStage::Enum stage)
{
ProfileEvents::increment(ProfileEvents::Query);
time_t current_time = time(0);
ParserQuery parser;
ASTPtr ast;
@ -67,13 +119,21 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
catch (...)
{
/// Всё равно логгируем запрос.
logQuery(String(begin, begin + std::min(end - begin, static_cast<ptrdiff_t>(max_query_size))), context);
if (!internal)
{
String query = String(begin, begin + std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
logQuery(query, context);
onExceptionBeforeStart(query, context, current_time);
}
throw;
}
String query(begin, query_size);
BlockIO res;
try
{
if (!internal)
logQuery(query, context);
@ -81,16 +141,15 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
checkLimits(*ast, context.getSettingsRef().limits);
QuotaForIntervals & quota = context.getQuota();
time_t current_time = time(0);
quota.checkExceeded(current_time);
const Settings & settings = context.getSettingsRef();
/// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем.
ProcessList::EntryPtr process_list_entry;
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
{
const Settings & settings = context.getSettingsRef();
process_list_entry = context.getProcessList().insert(
query, context.getUser(), context.getCurrentQueryId(), context.getIPAddress(),
settings.limits.max_memory_usage,
@ -101,23 +160,134 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
context.setProcessListElement(&process_list_entry->get());
}
BlockIO res;
try
{
auto interpreter = InterpreterFactory::get(ast, context, stage);
res = interpreter->execute();
/// Держим элемент списка процессов до конца обработки запроса.
res.process_list_entry = process_list_entry;
quota.addQuery(current_time);
/// Всё, что связано с логом запросов.
{
QueryLogElement elem;
elem.type = QueryLogElement::QUERY_START;
elem.event_time = current_time;
elem.query_start_time = current_time;
elem.query = query;
setClientInfo(elem, context);
bool log_queries = settings.log_queries;
/// Логгируем в таблицу начало выполнения запроса, если нужно.
if (log_queries)
context.getQueryLog().add(elem);
/// Также дадим вызывающему коду в дальнейшем логгировать завершение запроса и эксепшен.
res.finish_callback = [elem, &context, log_queries] (IBlockInputStream & stream) mutable
{
ProcessListElement * process_list_elem = context.getProcessListElement();
if (!process_list_elem)
return;
double elapsed_seconds = process_list_elem->watch.elapsedSeconds();
elem.type = QueryLogElement::QUERY_FINISH;
elem.event_time = time(0);
elem.query_duration_ms = elapsed_seconds * 1000;
elem.read_rows = process_list_elem->progress.rows;
elem.read_bytes = process_list_elem->progress.bytes;
auto memory_usage = process_list_elem->memory_tracker.getPeak();
elem.memory_usage = memory_usage > 0 ? memory_usage : 0;
if (IProfilingBlockInputStream * profiling_stream = dynamic_cast<IProfilingBlockInputStream *>(&stream))
{
const BlockStreamProfileInfo & info = profiling_stream->getInfo();
elem.result_rows = info.rows;
elem.result_bytes = info.bytes;
}
if (elem.read_rows != 0)
{
LOG_INFO(&Logger::get("executeQuery"), std::fixed << std::setprecision(3)
<< "Read " << elem.read_rows << " rows, "
<< formatReadableSizeWithBinarySuffix(elem.read_bytes) << " in " << elapsed_seconds << " sec., "
<< static_cast<size_t>(elem.read_rows / elapsed_seconds) << " rows/sec., "
<< formatReadableSizeWithBinarySuffix(elem.read_bytes / elapsed_seconds) << "/sec.");
}
if (log_queries)
context.getQueryLog().add(elem);
};
res.exception_callback = [elem, &context, log_queries, current_time] () mutable
{
context.getQuota().addError(current_time);
elem.type = QueryLogElement::EXCEPTION_WHILE_PROCESSING;
elem.event_time = time(0);
elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time);
elem.exception = getCurrentExceptionMessage(false);
ProcessListElement * process_list_elem = context.getProcessListElement();
if (process_list_elem)
{
double elapsed_seconds = process_list_elem->watch.elapsedSeconds();
elem.query_duration_ms = elapsed_seconds * 1000;
elem.read_rows = process_list_elem->progress.rows;
elem.read_bytes = process_list_elem->progress.bytes;
auto memory_usage = process_list_elem->memory_tracker.getPeak();
elem.memory_usage = memory_usage > 0 ? memory_usage : 0;
}
/// Достаём стек трейс, если возможно.
try
{
throw;
}
catch (const Exception & e)
{
elem.stack_trace = e.getStackTrace().toString();
LOG_ERROR(&Logger::get("executeQuery"), elem.exception << ", Stack trace:\n\n" << elem.stack_trace);
}
catch (...)
{
quota.addError(current_time);
throw;
LOG_ERROR(&Logger::get("executeQuery"), elem.exception);
}
quota.addQuery(current_time);
if (log_queries)
context.getQueryLog().add(elem);
};
if (!internal && res.in)
{
std::stringstream log_str;
log_str << "Query pipeline:\n";
res.in->dumpTree(log_str);
LOG_DEBUG(&Logger::get("executeQuery"), log_str.str());
}
}
}
catch (...)
{
onExceptionBeforeStart(query, context, current_time);
throw;
}
return std::make_tuple(ast, res);
}
@ -174,6 +344,8 @@ void executeQuery(
std::tie(ast, streams) = executeQueryImpl(begin, end, context, internal, stage);
try
{
if (streams.out)
{
const ASTInsertQuery * ast_insert_query = dynamic_cast<const ASTInsertQuery *>(ast.get());
@ -211,14 +383,22 @@ void executeQuery(
{
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
String format_name = ast_query_with_output && ast_query_with_output->format
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->format).name
String format_name = ast_query_with_output && (ast_query_with_output->getFormat() != nullptr)
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->getFormat()).name
: context.getDefaultFormat();
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, ostr, streams.in_sample);
copyData(*streams.in, *out);
}
}
catch (...)
{
streams.onException();
throw;
}
streams.onFinish();
}
}

View File

@ -0,0 +1,82 @@
#include <DB/Common/SipHash.h>
#include <DB/Parsers/ASTLiteral.h>
namespace DB
{
/** Обновляет SipHash по данным Field */
class FieldVisitorHash : public StaticVisitor<>
{
private:
SipHash & hash;
public:
FieldVisitorHash(SipHash & hash) : hash(hash) {}
void operator() (const Null & x) const
{
UInt8 type = Field::Types::Null;
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
}
void operator() (const UInt64 & x) const
{
UInt8 type = Field::Types::UInt64;
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
hash.update(reinterpret_cast<const char *>(&x), sizeof(x));
}
void operator() (const Int64 & x) const
{
UInt8 type = Field::Types::Int64;
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
hash.update(reinterpret_cast<const char *>(&x), sizeof(x));
}
void operator() (const Float64 & x) const
{
UInt8 type = Field::Types::Float64;
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
hash.update(reinterpret_cast<const char *>(&x), sizeof(x));
}
void operator() (const String & x) const
{
UInt8 type = Field::Types::String;
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
size_t size = x.size();
hash.update(reinterpret_cast<const char *>(&size), sizeof(size));
hash.update(x.data(), x.size());
}
void operator() (const Array & x) const
{
UInt8 type = Field::Types::Array;
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
size_t size = x.size();
hash.update(reinterpret_cast<const char *>(&size), sizeof(size));
for (const auto & elem : x)
apply_visitor(*this, elem);
}
};
String ASTLiteral::getColumnName() const
{
/// Отдельный случай для очень больших массивов. Вместо указания всех элементов, будем использовать хэш от содержимого.
if (value.getType() == Field::Types::Array
&& value.get<const Array &>().size() > 100) /// 100 - наугад.
{
SipHash hash;
apply_visitor(FieldVisitorHash(hash), value);
UInt64 low, high;
hash.get128(low, high);
return "__array_" + toString(low) + "_" + toString(high);
}
return apply_visitor(FieldVisitorToString(), value);
}
}

View File

@ -172,5 +172,14 @@ ASTPtr ASTSelectQuery::clone() const
return ptr;
}
const IAST * ASTSelectQuery::getFormat() const
{
const ASTSelectQuery * query = this;
while (!query->next_union_all.isNull())
query = static_cast<const ASTSelectQuery *>(query->next_union_all.get());
return query->format.get();
}
};

View File

@ -405,6 +405,59 @@ bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max
}
bool ParserArrayOfLiterals::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
Array arr;
if (pos == end || *pos != '[')
{
expected = "opening square bracket";
return false;
}
ParserWhiteSpaceOrComments ws;
ParserLiteral literal_p;
++pos;
while (pos != end)
{
ws.ignore(pos, end);
if (!arr.empty())
{
if (*pos == ']')
{
++pos;
node = new ASTLiteral(StringRange(begin, pos), arr);
return true;
}
else if (*pos == ',')
{
++pos;
}
else
{
expected = "comma or closing square bracket";
return false;
}
}
ws.ignore(pos, end);
ASTPtr literal_node;
if (!literal_p.parse(pos, end, literal_node, max_parsed_pos, expected))
return false;
arr.push_back(typeid_cast<const ASTLiteral &>(*literal_node).value);
}
expected = "closing square bracket";
return false;
}
bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
ParserNull null_p;
@ -450,6 +503,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
ParserParenthesisExpression paren_p;
ParserSubquery subquery_p;
ParserArray array_p;
ParserArrayOfLiterals array_lite_p;
ParserLiteral lit_p;
ParserFunction fun_p;
ParserCompoundIdentifier id_p;
@ -461,6 +515,9 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
if (paren_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
if (array_lite_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
if (array_p.parse(pos, end, node, max_parsed_pos, expected))
return true;

View File

@ -8,7 +8,6 @@
#include <DB/Parsers/ParserSetQuery.h>
#include <DB/Parsers/ParserSelectQuery.h>
namespace DB
{
@ -296,6 +295,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
ws.ignore(pos, end);
}
bool has_format = false;
/// FORMAT format_name
if (s_format.ignore(pos, end, max_parsed_pos, expected))
{
@ -308,6 +309,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
typeid_cast<ASTIdentifier &>(*select_query->format).kind = ASTIdentifier::Format;
ws.ignore(pos, end);
has_format = true;
}
// UNION ALL select query
@ -317,6 +319,13 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
if (s_all.ignore(pos, end, max_parsed_pos, expected))
{
if (has_format)
{
/// FORMAT может быть задан только в последнем запросе цепочки UNION ALL.
expected = "FORMAT only in the last SELECT of the UNION ALL chain";
return false;
}
ParserSelectQuery select_p;
if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected))
return false;

View File

@ -136,41 +136,24 @@ void HTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net
if (readonly)
context.getSettingsRef().limits.readonly = true;
Stopwatch watch;
context.setInterface(Context::Interface::HTTP);
Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN;
if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET)
http_method = Context::HTTPMethod::GET;
else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST)
http_method = Context::HTTPMethod::POST;
context.setHTTPMethod(http_method);
executeQuery(*in, *used_output.out_maybe_compressed, context, query_plan);
watch.stop();
if (query_plan)
{
std::stringstream log_str;
log_str << "Query pipeline:\n";
query_plan->dumpTree(log_str);
LOG_DEBUG(log, log_str.str());
/// Выведем информацию о том, сколько считано строк и байт.
size_t rows = 0;
size_t bytes = 0;
query_plan->getLeafRowsBytes(rows, bytes);
if (rows != 0)
{
LOG_INFO(log, std::fixed << std::setprecision(3)
<< "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., "
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.");
}
}
QuotaForIntervals & quota = context.getQuota();
if (!quota.empty())
LOG_INFO(log, "Quota:\n" << quota.toString());
/// Если не было эксепшена и данные ещё не отправлены - отправляются HTTP заголовки с кодом 200.
used_output.out->finalize();
}
void HTTPHandler::trySendExceptionToClient(std::stringstream & s,
void HTTPHandler::trySendExceptionToClient(const std::string & s,
Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response,
Output & used_output)
{
@ -191,7 +174,7 @@ void HTTPHandler::trySendExceptionToClient(std::stringstream & s,
if (!response.sent() && !used_output.out_maybe_compressed)
{
/// Ещё ничего не отправляли, и даже не знаем, нужно ли сжимать ответ.
response.send() << s.str() << std::endl;
response.send() << s << std::endl;
}
else if (used_output.out_maybe_compressed)
{
@ -207,8 +190,7 @@ void HTTPHandler::trySendExceptionToClient(std::stringstream & s,
used_output.out->position() = used_output.out->buffer().begin();
}
std::string exception_message = s.str();
writeString(exception_message, *used_output.out_maybe_compressed);
writeString(s, *used_output.out_maybe_compressed);
writeChar('\n', *used_output.out_maybe_compressed);
used_output.out_maybe_compressed->next();
used_output.out->finalize();
@ -245,35 +227,10 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
processQuery(request, response, used_output);
LOG_INFO(log, "Done processing query");
}
catch (Exception & e)
{
std::stringstream s;
s << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
<< ", Stack trace:\n\n" << e.getStackTrace().toString();
LOG_ERROR(log, s.str());
trySendExceptionToClient(s, request, response, used_output);
}
catch (Poco::Exception & e)
{
std::stringstream s;
s << "Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
LOG_ERROR(log, s.str());
trySendExceptionToClient(s, request, response, used_output);
}
catch (std::exception & e)
{
std::stringstream s;
s << "Code: " << ErrorCodes::STD_EXCEPTION << ". " << e.what();
LOG_ERROR(log, s.str());
trySendExceptionToClient(s, request, response, used_output);
}
catch (...)
{
std::stringstream s;
s << "Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ". Unknown exception.";
LOG_ERROR(log, s.str());
trySendExceptionToClient(s, request, response, used_output);
tryLogCurrentException(log);
trySendExceptionToClient(getCurrentExceptionMessage(true), request, response, used_output);
}
}

View File

@ -26,7 +26,7 @@ public:
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response);
void trySendExceptionToClient(std::stringstream & s,
void trySendExceptionToClient(const std::string & s,
Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response,
Output & used_output);

View File

@ -52,6 +52,9 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco:
context.setUser(user, password, request.clientAddress().host(), quota_key);
context.setInterface(Context::Interface::HTTP);
context.setHTTPMethod(Context::HTTPMethod::POST);
OLAP::QueryParseResult olap_query = server.olap_parser->parse(request_istream);
std::string clickhouse_query;
@ -66,31 +69,6 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco:
executeQuery(in, out, context, query_plan);
watch.stop();
if (query_plan)
{
std::stringstream log_str;
log_str << "Query pipeline:\n";
query_plan->dumpTree(log_str);
LOG_DEBUG(log, log_str.str());
/// Выведем информацию о том, сколько считано строк и байт.
size_t rows = 0;
size_t bytes = 0;
query_plan->getLeafRowsBytes(rows, bytes);
if (rows != 0)
{
LOG_INFO(log, std::fixed << std::setprecision(3)
<< "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., "
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.");
}
}
QuotaForIntervals & quota = context.getQuota();
if (!quota.empty())
LOG_INFO(log, "Quota:\n" << quota.toString());
/// Если не было эксепшена и данные ещё не отправлены - отправляются HTTP заголовки с кодом 200.
out.finalize();
}

View File

@ -533,7 +533,12 @@ int Server::main(const std::vector<std::string> & args)
LOG_DEBUG(log, "Loaded metadata.");
/// Создаём системные таблицы.
if (!global_context->isDatabaseExist("system"))
{
Poco::File(path + "data/system").createDirectories();
Poco::File(path + "metadata/system").createDirectories();
global_context->addDatabase("system");
}
global_context->addTable("system", "one", StorageSystemOne::create("one"));
global_context->addTable("system", "numbers", StorageSystemNumbers::create("numbers"));

View File

@ -111,6 +111,7 @@ void TCPHandler::runImpl()
{
/// Восстанавливаем контекст запроса.
query_context = connection_context;
query_context.setInterface(Context::Interface::TCP);
/** Если Query - обрабатываем. Если Ping или Cancel - возвращаемся в начало.
* Могут прийти настройки на отдельный запрос, которые модифицируют query_context.
@ -148,8 +149,7 @@ void TCPHandler::runImpl()
}
catch (const Exception & e)
{
LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
state.io.onException();
exception = e.clone();
if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT)
@ -163,24 +163,22 @@ void TCPHandler::runImpl()
* Хотя в одном из них, мы должны отправить эксепшен клиенту, а в другом - не можем.
* Будем пытаться отправить эксепшен клиенту в любом случае - см. ниже.
*/
LOG_ERROR(log, "Poco::Net::NetException. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
state.io.onException();
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION);
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
state.io.onException();
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION);
}
catch (const std::exception & e)
{
LOG_ERROR(log, "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", e.what() = " << e.what());
state.io.onException();
exception = new Exception(e.what(), ErrorCodes::STD_EXCEPTION);
}
catch (...)
{
LOG_ERROR(log, "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION);
state.io.onException();
exception = new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
}
@ -285,11 +283,6 @@ void TCPHandler::processOrdinaryQuery()
AsynchronousBlockInputStream async_in(state.io.in);
async_in.readPrefix();
std::stringstream query_pipeline;
async_in.dumpTree(query_pipeline);
LOG_DEBUG(log, "Query pipeline:\n" << query_pipeline.rdbuf());
Stopwatch watch;
while (true)
{
Block block;
@ -341,9 +334,7 @@ void TCPHandler::processOrdinaryQuery()
}
async_in.readSuffix();
watch.stop();
logProfileInfo(watch, *state.io.in);
state.io.onFinish();
}
}
@ -412,27 +403,6 @@ void TCPHandler::sendExtremes()
}
void TCPHandler::logProfileInfo(Stopwatch & watch, IBlockInputStream & in)
{
/// Выведем информацию о том, сколько считано строк и байт.
size_t rows = 0;
size_t bytes = 0;
in.getLeafRowsBytes(rows, bytes);
if (rows != 0)
{
LOG_INFO(log, std::fixed << std::setprecision(3)
<< "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., "
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.");
}
QuotaForIntervals & quota = query_context.getQuota();
if (!quota.empty())
LOG_INFO(log, "Quota:\n" << quota.toString());
}
void TCPHandler::receiveHello()
{
/// Получить hello пакет.

View File

@ -130,9 +130,6 @@ private:
/// Эта функция вызывается из разных потоков.
void updateProgress(const Progress & value);
/// Вывести информацию о скорости выполнения SELECT запроса.
void logProfileInfo(Stopwatch & watch, IBlockInputStream & in);
};

View File

@ -11,7 +11,6 @@
#include <DB/IO/CompressedReadBuffer.h>
#include <DB/IO/HashingReadBuffer.h>
#include <DB/Columns/ColumnsNumber.h>
#include <statdaemons/ext/scope_guard.hpp>
namespace DB
@ -116,12 +115,6 @@ struct Stream
readIntBinary(mrk_mark.offset_in_compressed_file, mrk_hashing_buf);
readIntBinary(mrk_mark.offset_in_decompressed_block, mrk_hashing_buf);
/// На всякий случай, сохраним смещение в файле и размер предыдущего блока.
SCOPE_EXIT(
prev_offset_in_compressed_file = mrk_mark.offset_in_compressed_file;
prev_buffer_size = uncompressed_hashing_buf.buffer().size();
);
bool has_alternative_mark = false;
MarkInCompressedFile alternative_data_mark;
MarkInCompressedFile data_mark;
@ -145,18 +138,6 @@ struct Stream
if (uncompressed_hashing_buf.eof())
return;
}
else if (uncompressed_hashing_buf.offset() == 0)
{
/// Восстановим засечку на конец предыдущего блока по сохраненным данным
has_alternative_mark = true;
alternative_data_mark.offset_in_compressed_file = prev_offset_in_compressed_file;
alternative_data_mark.offset_in_decompressed_block = prev_buffer_size;
if (mrk_mark == alternative_data_mark)
return;
}
std::cout << "mrk_mark " << mrk_mark.offset_in_compressed_file << ' ' << mrk_mark.offset_in_decompressed_block << std::endl;
data_mark.offset_in_compressed_file = compressed_hashing_buf.count() - uncompressing_buf.getSizeCompressed();
data_mark.offset_in_decompressed_block = uncompressed_hashing_buf.offset();
@ -180,10 +161,6 @@ struct Stream
checksums.files[name + ".mrk"] = MergeTreeData::DataPart::Checksums::Checksum(
mrk_hashing_buf.count(), mrk_hashing_buf.getHash());
}
private:
size_t prev_offset_in_compressed_file{};
size_t prev_buffer_size{};
};
/// Возвращает количество строк. Добавляет в checksums чексуммы всех файлов столбца.

View File

@ -8,6 +8,136 @@
namespace DB
{
/// Преобразование строки с датой или датой-с-временем в UInt64, содержащим числовое значение даты или даты-с-временем.
UInt64 stringToDateOrDateTime(const String & s)
{
ReadBufferFromString in(s);
if (s.size() == strlen("YYYY-MM-DD"))
{
DayNum_t date{};
readDateText(date, in);
return UInt64(date);
}
else
{
time_t date_time{};
readDateTimeText(date_time, in);
if (!in.eof())
throw Exception("String is too long for DateTime: " + s);
return UInt64(date_time);
}
}
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
/** Более точное сравнение.
* Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой.
* Правила сравнения - такие же, что и в FunctionsComparison.
* В том числе, сравнение знаковых и беззнаковых оставляем UB.
*/
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
{
public:
bool operator() (const Null & l, const Null & r) const { return true; }
bool operator() (const Null & l, const UInt64 & r) const { return false; }
bool operator() (const Null & l, const Int64 & r) const { return false; }
bool operator() (const Null & l, const Float64 & r) const { return false; }
bool operator() (const Null & l, const String & r) const { return false; }
bool operator() (const Null & l, const Array & r) const { return false; }
bool operator() (const UInt64 & l, const Null & r) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const String & r) const { return l == stringToDateOrDateTime(r); }
bool operator() (const UInt64 & l, const Array & r) const { return false; }
bool operator() (const Int64 & l, const Null & r) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
bool operator() (const Int64 & l, const Float64 & r) const { return l == r; }
bool operator() (const Int64 & l, const String & r) const { return false; }
bool operator() (const Int64 & l, const Array & r) const { return false; }
bool operator() (const Float64 & l, const Null & r) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const Float64 & l, const Int64 & r) const { return l == r; }
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
bool operator() (const Float64 & l, const String & r) const { return false; }
bool operator() (const Float64 & l, const Array & r) const { return false; }
bool operator() (const String & l, const Null & r) const { return false; }
bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) == r; }
bool operator() (const String & l, const Int64 & r) const { return false; }
bool operator() (const String & l, const Float64 & r) const { return false; }
bool operator() (const String & l, const String & r) const { return l == r; }
bool operator() (const String & l, const Array & r) const { return false; }
bool operator() (const Array & l, const Null & r) const { return false; }
bool operator() (const Array & l, const UInt64 & r) const { return false; }
bool operator() (const Array & l, const Int64 & r) const { return false; }
bool operator() (const Array & l, const Float64 & r) const { return false; }
bool operator() (const Array & l, const String & r) const { return false; }
bool operator() (const Array & l, const Array & r) const { return l == r; }
};
class FieldVisitorAccurateLess : public StaticVisitor<bool>
{
public:
bool operator() (const Null & l, const Null & r) const { return false; }
bool operator() (const Null & l, const UInt64 & r) const { return true; }
bool operator() (const Null & l, const Int64 & r) const { return true; }
bool operator() (const Null & l, const Float64 & r) const { return true; }
bool operator() (const Null & l, const String & r) const { return true; }
bool operator() (const Null & l, const Array & r) const { return true; }
bool operator() (const UInt64 & l, const Null & r) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const String & r) const { return l < stringToDateOrDateTime(r); }
bool operator() (const UInt64 & l, const Array & r) const { return true; }
bool operator() (const Int64 & l, const Null & r) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
bool operator() (const Int64 & l, const Float64 & r) const { return l < r; }
bool operator() (const Int64 & l, const String & r) const { return true; }
bool operator() (const Int64 & l, const Array & r) const { return true; }
bool operator() (const Float64 & l, const Null & r) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const Float64 & l, const Int64 & r) const { return l < r; }
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
bool operator() (const Float64 & l, const String & r) const { return true; }
bool operator() (const Float64 & l, const Array & r) const { return true; }
bool operator() (const String & l, const Null & r) const { return false; }
bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) < r; }
bool operator() (const String & l, const Int64 & r) const { return false; }
bool operator() (const String & l, const Float64 & r) const { return false; }
bool operator() (const String & l, const String & r) const { return l < r; }
bool operator() (const String & l, const Array & r) const { return true; }
bool operator() (const Array & l, const Null & r) const { return false; }
bool operator() (const Array & l, const UInt64 & r) const { return false; }
bool operator() (const Array & l, const Int64 & r) const { return false; }
bool operator() (const Array & l, const Float64 & r) const { return false; }
bool operator() (const Array & l, const String & r) const { return false; }
bool operator() (const Array & l, const Array & r) const { return l < r; }
};
#pragma GCC diagnostic pop
inline bool Range::equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
inline bool Range::less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_)
: sort_descr(sort_descr_)
{

View File

@ -545,7 +545,8 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
{
LOG_INFO(log, "Shutdown requested while merging chunks.");
output->writeSuffix();
new_storage.removeReference(); /// После этого временные данные удалятся.
output = nullptr;
executeQuery("DROP TABLE IF EXISTS " + new_table_full_name, context, true);
return false;
}
@ -575,14 +576,16 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
/// Отцепляем исходную таблицу. Ее данные и метаданные остаются на диске.
tables_to_drop.push_back(context.detachTable(chunk_merger.source_database, src_name));
/// Создаем на ее месте ChunkRef. Это возможно только потому что у ChunkRef нет ни, ни метаданных.
/// Создаем на ее месте ChunkRef. Это возможно только потому что у ChunkRef нет ни данных, ни метаданных.
try
{
context.addTable(chunk_merger.source_database, src_name, StorageChunkRef::create(src_name, context, chunk_merger.source_database, new_table_name, false));
context.addTable(chunk_merger.source_database, src_name,
StorageChunkRef::create(src_name, context, chunk_merger.source_database, new_table_name, false));
}
catch (...)
{
LOG_ERROR(log, "Chunk " + src_name + " was removed but not replaced. Its data is stored in table " << new_table_name << ". You may need to resolve this manually.");
LOG_ERROR(log, "Chunk " + src_name + " was removed but not replaced. Its data is stored in table "
<< new_table_name << ". You may need to resolve this manually.");
throw;
}
@ -601,9 +604,6 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
/// что-нибудь может сломаться.
}
/// Сейчас на new_storage ссылаются таблицы типа ChunkRef. Удалим лишнюю ссылку, которая была при создании.
new_storage.removeReference();
LOG_TRACE(log, "Merged chunks.");
return true;
@ -613,6 +613,7 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
currently_written_groups.erase(new_table_full_name);
executeQuery("DROP TABLE IF EXISTS " + new_table_full_name, context, true);
throw;
}

View File

@ -161,9 +161,6 @@ StorageChunks::StorageChunks(
context(context_),
log(&Logger::get("StorageChunks"))
{
if (!attach)
reference_counter.add(1, true);
_table_column_name = "_table" + VirtualColumnUtils::chooseSuffix(getColumnsList(), "_table");
try

View File

@ -154,7 +154,7 @@ BlockInputStreams StorageDistributed::read(
size_t result_size = (cluster.pools.size() * settings.max_parallel_replicas) + cluster.getLocalNodesNum();
processed_stage = result_size == 1
processed_stage = result_size == 1 || settings.distributed_group_by_no_merge
? QueryProcessingStage::Complete
: QueryProcessingStage::WithMergeableState;

View File

@ -233,7 +233,15 @@ Block LogBlockInputStream::readImpl()
else
column.column = column.type->createColumn();
try
{
readData(*it, *column.type, *column.column, max_rows_to_read, 0, read_offsets);
}
catch (Exception & e)
{
e.addMessage("while reading column " + *it + " at " + storage.path + escapeForFileName(storage.name));
throw;
}
if (column.column->size())
res.insert(column);

View File

@ -198,7 +198,15 @@ Block TinyLogBlockInputStream::readImpl()
else
column.column = column.type->createColumn();
try
{
readData(*it, *column.type, *column.column, block_size, 0, read_offsets);
}
catch (Exception & e)
{
e.addMessage("while reading column " + *it + " at " + storage.full_path());
throw;
}
if (column.column->size())
res.insert(column);

View File

@ -0,0 +1,18 @@
1
1
1
1
1
1
1
1
1
1
1
1

View File

@ -0,0 +1,3 @@
SELECT 1 FORMAT PrettySpace;
SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;

View File

@ -0,0 +1,12 @@
DROP TABLE IF EXISTS test.t;
DROP TABLE IF EXISTS test.mv;
DROP TABLE IF EXISTS test.`.inner.mv`;
CREATE TABLE test.t (x UInt8) ENGINE = Null;
CREATE MATERIALIZED VIEW test.mv AS SELECT * FROM test.t;
DETACH TABLE test.mv;
ATTACH MATERIALIZED VIEW test.mv AS SELECT * FROM test.t;
DROP TABLE test.t;
DROP TABLE test.mv;

View File

@ -0,0 +1,14 @@
inf
0
inf
0
0
0
0
0
inf
0
0
0
inf
0

View File

@ -0,0 +1,143 @@
DROP TABLE IF EXISTS series;
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
/* Тестовые данные */
INSERT INTO series(i, x_value, y_value) VALUES (1, 5.6,-4.4),(2, -9.6,3),(3, -1.3,-4),(4, 5.3,9.7),(5, 4.4,0.037),(6, -8.6,-7.8),(7, 5.1,9.3),(8, 7.9,-3.6),(9, -8.2,0.62),(10, -3,7.3);
/* varSamp */
SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
varSamp(x_value) AS res1,
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1) AS res2
FROM series
);
/* stddevSamp */
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
stddevSamp(x_value) AS res1,
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1)) AS res2
FROM series
);
/* varPop */
SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
varPop(x_value) AS res1,
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count() AS res2
FROM series
);
/* stddevPop */
SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
stddevPop(x_value) AS res1,
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count()) AS res2
FROM series
);
/* covarSamp */
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(COVAR1 - COVAR2), 6)
FROM
(
SELECT
arrayJoin([1]) AS ID2,
covarSamp(x_value, y_value) AS COVAR1
FROM series
) ANY INNER JOIN
(
SELECT
arrayJoin([1]) AS ID2,
sum(VAL) / (count() - 1) AS COVAR2
FROM
(
SELECT (X - AVG_X) * (Y - AVG_Y) AS VAL
FROM
(
SELECT
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
avg(x_value) AS AVG_X,
avg(y_value) AS AVG_Y
FROM series
) ANY INNER JOIN
(
SELECT
i AS ID,
x_value AS X,
y_value AS Y
FROM series
) USING ID
)
) USING ID2;
/* covarPop */
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(COVAR1 - COVAR2), 6)
FROM
(
SELECT
arrayJoin([1]) AS ID2,
covarPop(x_value, y_value) AS COVAR1
FROM series
) ANY INNER JOIN
(
SELECT
arrayJoin([1]) AS ID2,
sum(VAL) / count() AS COVAR2
FROM
(
SELECT (X - AVG_X) * (Y - AVG_Y) AS VAL
FROM
(
SELECT
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
avg(x_value) AS AVG_X,
avg(y_value) AS AVG_Y
FROM series
) ANY INNER JOIN
(
SELECT
i AS ID,
x_value AS X,
y_value AS Y
FROM series
) USING ID
)
) USING ID2;
/* corr */
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(corr(x_value, y_value) - covarPop(x_value, y_value) / (stddevPop(x_value) * stddevPop(y_value))), 6) FROM series;

View File

@ -0,0 +1,914 @@
---map--
[]
[123,123,123]
[]
[123]
[123,123]
[123,123,123]
[123,123,123,123]
[123,123,123,123,123]
[123,123,123,123,123,123]
[123,123,123,123,123,123,123]
[123,123,123,123,123,123,123,123]
[123,123,123,123,123,123,123,123,123]
---filter--
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[1,2,3]
[]
[0]
[0,1]
[0,1,2]
[0,1,2,3]
[0,1,2,3,4]
[0,1,2,3,4,5]
[0,1,2,3,4,5,6]
[0,1,2,3,4,5,6,7]
[0,1,2,3,4,5,6,7,8]
---count---
0
0
0
0
0
0
0
0
0
0
0
0
0
3
0
1
2
3
4
5
6
7
8
9
---sum---
0
0
0
0
0
0
0
0
0
0
0
0
0
30
0
10
20
30
40
50
60
70
80
90
---all---
1
0
1
0
0
0
0
0
0
0
0
0
1
1
1
1
1
1
1
1
1
1
1
1
---exists---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
1
1
1
1
1
1
1
1
1
---first---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
0
0
0
0
0
0
0
0
0
---first index---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
1
1
1
1
1
1
1
1
1
---map--
[]
[123,123,123]
[]
[123]
[123,123]
[123,123,123]
[123,123,123,123]
[123,123,123,123,123]
[123,123,123,123,123,123]
[123,123,123,123,123,123,123]
[123,123,123,123,123,123,123,123]
[123,123,123,123,123,123,123,123,123]
---filter--
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[1,2,3]
[]
[0]
[0,1]
[0,1,2]
[0,1,2,3]
[0,1,2,3,4]
[0,1,2,3,4,5]
[0,1,2,3,4,5,6]
[0,1,2,3,4,5,6,7]
[0,1,2,3,4,5,6,7,8]
---count---
0
0
0
0
0
0
0
0
0
0
0
0
0
3
0
1
2
3
4
5
6
7
8
9
---sum---
0
0
0
0
0
0
0
0
0
0
0
0
0
30
0
10
20
30
40
50
60
70
80
90
---all---
1
0
1
0
0
0
0
0
0
0
0
0
1
1
1
1
1
1
1
1
1
1
1
1
---exists---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
1
1
1
1
1
1
1
1
1
---first---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
0
0
0
0
0
0
0
0
0
---first index---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
1
1
1
1
1
1
1
1
1
---map--
[]
[123,123,123]
[]
[123]
[123,123]
[123,123,123]
[123,123,123,123]
[123,123,123,123,123]
[123,123,123,123,123,123]
[123,123,123,123,123,123,123]
[123,123,123,123,123,123,123,123]
[123,123,123,123,123,123,123,123,123]
---filter--
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
['1','2','3']
[]
['0']
['0','1']
['0','1','2']
['0','1','2','3']
['0','1','2','3','4']
['0','1','2','3','4','5']
['0','1','2','3','4','5','6']
['0','1','2','3','4','5','6','7']
['0','1','2','3','4','5','6','7','8']
---count---
0
0
0
0
0
0
0
0
0
0
0
0
0
3
0
1
2
3
4
5
6
7
8
9
---sum---
0
0
0
0
0
0
0
0
0
0
0
0
0
30
0
10
20
30
40
50
60
70
80
90
---all---
1
0
1
0
0
0
0
0
0
0
0
0
1
1
1
1
1
1
1
1
1
1
1
1
---exists---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
1
1
1
1
1
1
1
1
1
---first---
1
0
0
0
0
0
0
0
0
0
---first index---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
1
1
1
1
1
1
1
1
1
---map--
[]
[123,123,123]
[]
[123]
[123,123]
[123,123,123]
[123,123,123,123]
[123,123,123,123,123]
[123,123,123,123,123,123]
[123,123,123,123,123,123,123]
[123,123,123,123,123,123,123,123]
[123,123,123,123,123,123,123,123,123]
---filter--
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
[]
['1','2','3']
[]
['0']
['0','1']
['0','1','2']
['0','1','2','3']
['0','1','2','3','4']
['0','1','2','3','4','5']
['0','1','2','3','4','5','6']
['0','1','2','3','4','5','6','7']
['0','1','2','3','4','5','6','7','8']
---count---
0
0
0
0
0
0
0
0
0
0
0
0
0
3
0
1
2
3
4
5
6
7
8
9
---sum---
0
0
0
0
0
0
0
0
0
0
0
0
0
30
0
10
20
30
40
50
60
70
80
90
---all---
1
0
1
0
0
0
0
0
0
0
0
0
1
1
1
1
1
1
1
1
1
1
1
1
---exists---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
1
1
1
1
1
1
1
1
1
---first---
1
0
0
0
0
0
0
0
0
0
---first index---
0
0
0
0
0
0
0
0
0
0
0
0
0
1
0
1
1
1
1
1
1
1
1
1
--- ---
[]
[1]
[0,0]
[1,1,1]
[0,0,0,0]
[1,1,1,1,1]
[0,0,0,0,0,0]
[1,1,1,1,1,1,1]
[0,0,0,0,0,0,0,0]
[1,1,1,1,1,1,1,1,1]
[]
[0]
[]
[0,1,2]
[]
[0,1,2,3,4]
[]
[0,1,2,3,4,5,6]
[]
[0,1,2,3,4,5,6,7,8]
0
1
0
3
0
5
0
7
0
9
0
1
0
3
0
5
0
7
0
9
1
1
0
1
0
1
0
1
0
1
0
1
0
1
0
1
0
1
0
1
0
0
0
0
0
0
0
0
0
0
0
1
0
1
0
1
0
1
0
1
--- ---
[]
[1]
[0,0]
[1,1,1]
[0,0,0,0]
[1,1,1,1,1]
[0,0,0,0,0,0]
[1,1,1,1,1,1,1]
[0,0,0,0,0,0,0,0]
[1,1,1,1,1,1,1,1,1]
[]
['0']
[]
['0','1','2']
[]
['0','1','2','3','4']
[]
['0','1','2','3','4','5','6']
[]
['0','1','2','3','4','5','6','7','8']
0
1
0
3
0
5
0
7
0
9
0
1
0
3
0
5
0
7
0
9
1
1
0
1
0
1
0
1
0
1
0
1
0
1
0
1
0
1
0
1
0
0
0
0
0
0
1
0
1
0
1
0
1
0
1

View File

@ -0,0 +1,235 @@
SELECT '---map--';
SELECT arrayMap(x -> 123, emptyArrayUInt8());
SELECT arrayMap(x -> 123, [1, 2, 3]);
SELECT arrayMap(x -> 123, range(number)) FROM system.numbers LIMIT 10;
SELECT '---filter--';
SELECT arrayFilter(x -> 0, emptyArrayUInt8());
SELECT arrayFilter(x -> 0, [1, 2, 3]);
SELECT arrayFilter(x -> 0, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFilter(x -> 1, emptyArrayUInt8());
SELECT arrayFilter(x -> 1, [1, 2, 3]);
SELECT arrayFilter(x -> 1, range(number)) FROM system.numbers LIMIT 10;
SELECT '---count---';
SELECT arrayCount(x -> 0, emptyArrayUInt8());
SELECT arrayCount(x -> 0, [1, 2, 3]);
SELECT arrayCount(x -> 0, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayCount(x -> 1, emptyArrayUInt8());
SELECT arrayCount(x -> 1, [1, 2, 3]);
SELECT arrayCount(x -> 1, range(number)) FROM system.numbers LIMIT 10;
SELECT '---sum---';
SELECT arraySum(x -> 0, emptyArrayUInt8());
SELECT arraySum(x -> 0, [1, 2, 3]);
SELECT arraySum(x -> 0, range(number)) FROM system.numbers LIMIT 10;
SELECT arraySum(x -> 10, emptyArrayUInt8());
SELECT arraySum(x -> 10, [1, 2, 3]);
SELECT arraySum(x -> 10, range(number)) FROM system.numbers LIMIT 10;
SELECT '---all---';
SELECT arrayAll(x -> 0, emptyArrayUInt8());
SELECT arrayAll(x -> 0, [1, 2, 3]);
SELECT arrayAll(x -> 0, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayAll(x -> 1, emptyArrayUInt8());
SELECT arrayAll(x -> 1, [1, 2, 3]);
SELECT arrayAll(x -> 1, range(number)) FROM system.numbers LIMIT 10;
SELECT '---exists---';
SELECT arrayExists(x -> 0, emptyArrayUInt8());
SELECT arrayExists(x -> 0, [1, 2, 3]);
SELECT arrayExists(x -> 0, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayExists(x -> 1, emptyArrayUInt8());
SELECT arrayExists(x -> 1, [1, 2, 3]);
SELECT arrayExists(x -> 1, range(number)) FROM system.numbers LIMIT 10;
SELECT '---first---';
SELECT arrayFirst(x -> 0, emptyArrayUInt8());
SELECT arrayFirst(x -> 0, [1, 2, 3]);
SELECT arrayFirst(x -> 0, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFirst(x -> 1, emptyArrayUInt8());
SELECT arrayFirst(x -> 1, [1, 2, 3]);
SELECT arrayFirst(x -> 1, range(number)) FROM system.numbers LIMIT 10;
SELECT '---first index---';
SELECT arrayFirstIndex(x -> 0, emptyArrayUInt8());
SELECT arrayFirstIndex(x -> 0, [1, 2, 3]);
SELECT arrayFirstIndex(x -> 0, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFirstIndex(x -> 1, emptyArrayUInt8());
SELECT arrayFirstIndex(x -> 1, [1, 2, 3]);
SELECT arrayFirstIndex(x -> 1, range(number)) FROM system.numbers LIMIT 10;
SELECT '---map--';
SELECT arrayMap(x -> materialize(123), emptyArrayUInt8());
SELECT arrayMap(x -> materialize(123), [1, 2, 3]);
SELECT arrayMap(x -> materialize(123), range(number)) FROM system.numbers LIMIT 10;
SELECT '---filter--';
SELECT arrayFilter(x -> materialize(0), emptyArrayUInt8());
SELECT arrayFilter(x -> materialize(0), [1, 2, 3]);
SELECT arrayFilter(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFilter(x -> materialize(1), emptyArrayUInt8());
SELECT arrayFilter(x -> materialize(1), [1, 2, 3]);
SELECT arrayFilter(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
SELECT '---count---';
SELECT arrayCount(x -> materialize(0), emptyArrayUInt8());
SELECT arrayCount(x -> materialize(0), [1, 2, 3]);
SELECT arrayCount(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
SELECT arrayCount(x -> materialize(1), emptyArrayUInt8());
SELECT arrayCount(x -> materialize(1), [1, 2, 3]);
SELECT arrayCount(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
SELECT '---sum---';
SELECT arraySum(x -> materialize(0), emptyArrayUInt8());
SELECT arraySum(x -> materialize(0), [1, 2, 3]);
SELECT arraySum(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
SELECT arraySum(x -> materialize(10), emptyArrayUInt8());
SELECT arraySum(x -> materialize(10), [1, 2, 3]);
SELECT arraySum(x -> materialize(10), range(number)) FROM system.numbers LIMIT 10;
SELECT '---all---';
SELECT arrayAll(x -> materialize(0), emptyArrayUInt8());
SELECT arrayAll(x -> materialize(0), [1, 2, 3]);
SELECT arrayAll(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
SELECT arrayAll(x -> materialize(1), emptyArrayUInt8());
SELECT arrayAll(x -> materialize(1), [1, 2, 3]);
SELECT arrayAll(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
SELECT '---exists---';
SELECT arrayExists(x -> materialize(0), emptyArrayUInt8());
SELECT arrayExists(x -> materialize(0), [1, 2, 3]);
SELECT arrayExists(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
SELECT arrayExists(x -> materialize(1), emptyArrayUInt8());
SELECT arrayExists(x -> materialize(1), [1, 2, 3]);
SELECT arrayExists(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
SELECT '---first---';
SELECT arrayFirst(x -> materialize(0), emptyArrayUInt8());
SELECT arrayFirst(x -> materialize(0), [1, 2, 3]);
SELECT arrayFirst(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFirst(x -> materialize(1), emptyArrayUInt8());
SELECT arrayFirst(x -> materialize(1), [1, 2, 3]);
SELECT arrayFirst(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
SELECT '---first index---';
SELECT arrayFirstIndex(x -> materialize(0), emptyArrayUInt8());
SELECT arrayFirstIndex(x -> materialize(0), [1, 2, 3]);
SELECT arrayFirstIndex(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFirstIndex(x -> materialize(1), emptyArrayUInt8());
SELECT arrayFirstIndex(x -> materialize(1), [1, 2, 3]);
SELECT arrayFirstIndex(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
SELECT '---map--';
SELECT arrayMap(x -> 123, emptyArrayString());
SELECT arrayMap(x -> 123, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayMap(x -> 123, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---filter--';
SELECT arrayFilter(x -> 0, emptyArrayString());
SELECT arrayFilter(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFilter(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFilter(x -> 1, emptyArrayString());
SELECT arrayFilter(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFilter(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---count---';
SELECT arrayCount(x -> 0, emptyArrayString());
SELECT arrayCount(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayCount(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayCount(x -> 1, emptyArrayString());
SELECT arrayCount(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayCount(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---sum---';
SELECT arraySum(x -> 0, emptyArrayString());
SELECT arraySum(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arraySum(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arraySum(x -> 10, emptyArrayString());
SELECT arraySum(x -> 10, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arraySum(x -> 10, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---all---';
SELECT arrayAll(x -> 0, emptyArrayString());
SELECT arrayAll(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayAll(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayAll(x -> 1, emptyArrayString());
SELECT arrayAll(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayAll(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---exists---';
SELECT arrayExists(x -> 0, emptyArrayString());
SELECT arrayExists(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayExists(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayExists(x -> 1, emptyArrayString());
SELECT arrayExists(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayExists(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---first---';
SELECT arrayFirst(x -> 0, emptyArrayString());
SELECT arrayFirst(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFirst(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFirst(x -> 1, emptyArrayString());
SELECT arrayFirst(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFirst(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---first index---';
SELECT arrayFirstIndex(x -> 0, emptyArrayString());
SELECT arrayFirstIndex(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFirstIndex(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFirstIndex(x -> 1, emptyArrayString());
SELECT arrayFirstIndex(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFirstIndex(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---map--';
SELECT arrayMap(x -> materialize(123), emptyArrayString());
SELECT arrayMap(x -> materialize(123), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayMap(x -> materialize(123), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---filter--';
SELECT arrayFilter(x -> materialize(0), emptyArrayString());
SELECT arrayFilter(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFilter(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFilter(x -> materialize(1), emptyArrayString());
SELECT arrayFilter(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFilter(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---count---';
SELECT arrayCount(x -> materialize(0), emptyArrayString());
SELECT arrayCount(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayCount(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayCount(x -> materialize(1), emptyArrayString());
SELECT arrayCount(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayCount(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---sum---';
SELECT arraySum(x -> materialize(0), emptyArrayString());
SELECT arraySum(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arraySum(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arraySum(x -> materialize(10), emptyArrayString());
SELECT arraySum(x -> materialize(10), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arraySum(x -> materialize(10), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---all---';
SELECT arrayAll(x -> materialize(0), emptyArrayString());
SELECT arrayAll(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayAll(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayAll(x -> materialize(1), emptyArrayString());
SELECT arrayAll(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayAll(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---exists---';
SELECT arrayExists(x -> materialize(0), emptyArrayString());
SELECT arrayExists(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayExists(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayExists(x -> materialize(1), emptyArrayString());
SELECT arrayExists(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayExists(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---first---';
SELECT arrayFirst(x -> materialize(0), emptyArrayString());
SELECT arrayFirst(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFirst(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFirst(x -> materialize(1), emptyArrayString());
SELECT arrayFirst(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFirst(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '---first index---';
SELECT arrayFirstIndex(x -> materialize(0), emptyArrayString());
SELECT arrayFirstIndex(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFirstIndex(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFirstIndex(x -> materialize(1), emptyArrayString());
SELECT arrayFirstIndex(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
SELECT arrayFirstIndex(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT '--- ---';
SELECT arrayMap(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFilter(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayCount(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
SELECT arraySum(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayAll(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayExists(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFirst(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
SELECT arrayFirstIndex(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
SELECT '--- ---';
SELECT arrayMap(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFilter(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayCount(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arraySum(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayAll(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayExists(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFirst(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
SELECT arrayFirstIndex(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;

View File

@ -0,0 +1 @@
SELECT count() FROM remote('{127,1}.0.0.{1,2}', system.one) SETTINGS skip_unavailable_shards = 1;

View File

@ -0,0 +1 @@
SELECT count(), uniq(dummy) FROM remote('127.0.0.{1,2}', system.one) SETTINGS distributed_group_by_no_merge = 1;

View File

@ -0,0 +1,23 @@
[1,2]
[1,2]
[-1,2]
[-1,1000]
[-1,1000000]
[-1000,2]
[-1000000,2]
['Hello','world']
[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256]
[1,2]
[1,2]
[-1,2]
[-1,1000]
[-1,1000000]
[-1000,2]
[-1000000,2]
['Hello','world']
[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256]
[0]
[0]
[0]
[0]
[0]

View File

@ -0,0 +1,30 @@
SELECT [1, 2];
SELECT [1.0, 2];
SELECT [-1, 2];
SELECT [-1, 1000];
SELECT [-1, 1000000];
SELECT [-1000, 2];
SELECT [-1000000, 2];
SELECT ['Hello', 'world'];
SELECT [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256];
SELECT [1, 1 + 1];
SELECT [1.0, 1 + 1];
SELECT [-1, 1 + 1];
SELECT [-1, toUInt16(1000)];
SELECT [-1, toUInt32(1000000)];
SELECT [-1000, 1 + 1];
SELECT [-1000000, 1 + 1];
SELECT ['Hello', concat('wor', 'ld')];
SELECT [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256 + 0];
SELECT [0];
SELECT [0 ];
SELECT [ 0];
SELECT [ 0 ];
SELECT
[
0
];

View File

@ -0,0 +1 @@
10000000

View File

@ -0,0 +1,3 @@
#!/bin/bash
(echo 'SELECT number FROM system.numbers WHERE transform(number, ['; seq 1 100000 | tr '\n' ','; echo '0],['; seq 1 100000 | tr '\n' ','; echo '0]) = 10000000 LIMIT 1';) | clickhouse-client --max_query_size=100000000

View File

@ -0,0 +1 @@
SELECT materialize('prepre_f') LIKE '%pre_f%';

View File

@ -0,0 +1 @@
SELECT count(), sum(1), uniq(123) FROM (SELECT * FROM system.numbers LIMIT 10);

View File

@ -47,7 +47,6 @@ public:
</zookeeper>
*/
ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name);
ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, int32_t session_timeout_ms);
~ZooKeeper();
@ -126,6 +125,9 @@ public:
void set(const std::string & path, const std::string & data,
int32_t version = -1, Stat * stat = nullptr);
/** Создает ноду, если ее не существует. Иначе обновляет */
void createOrUpdate(const std::string & path, const std::string & data, int32_t mode);
/** Не бросает исключение при следующих ошибках:
* - Такой ноды нет.
* - У ноды другая версия.

View File

@ -128,13 +128,6 @@ ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std
init(args.hosts, args.session_timeout_ms);
}
ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration& config, const std::string& config_name, int32_t session_timeout_ms_)
{
ZooKeeperArgs args(config, config_name);
init(args.hosts, session_timeout_ms_);
}
void * ZooKeeper::watchForEvent(EventPtr event)
{
if (event)
@ -390,7 +383,7 @@ bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat_
{
int32_t code = retry(std::bind(&ZooKeeper::getImpl, this, std::ref(path), std::ref(res), stat_, watch));
if (!( code == ZOK ||
if (!(code == ZOK ||
code == ZNONODE))
throw KeeperException(code, path);
@ -418,6 +411,17 @@ void ZooKeeper::set(const std::string & path, const std::string & data, int32_t
check(trySet(path, data, version, stat), path);
}
void ZooKeeper::createOrUpdate(const std::string & path, const std::string & data, int32_t mode)
{
int code = trySet(path, data, -1);
if (code == ZNONODE)
{
create(path, data, mode);
}
else if (code != ZOK)
throw zkutil::KeeperException(code, path);
}
int32_t ZooKeeper::trySet(const std::string & path, const std::string & data,
int32_t version, Stat * stat_)
{