dbms: Server: merge from master [#METR-15090]

This commit is contained in:
Alexey Arno 2015-03-04 15:15:09 +03:00
commit e6a1a112a5
46 changed files with 1738 additions and 1171 deletions

View File

@ -160,12 +160,17 @@ namespace detail
void insert(UInt64 x)
{
++count;
insertWeighted(x, 1);
}
void insertWeighted(UInt64 x, size_t weight)
{
count += weight;
if (x < SMALL_THRESHOLD)
++count_small[x];
count_small[x] += weight;
else if (x < BIG_THRESHOLD)
++count_big[(x - SMALL_THRESHOLD) / BIG_PRECISION];
count_big[(x - SMALL_THRESHOLD) / BIG_PRECISION] += weight;
}
void merge(const QuantileTimingLarge & rhs)
@ -371,6 +376,23 @@ public:
}
}
void insertWeighted(UInt64 x, size_t weight)
{
/// NOTE: Первое условие - для того, чтобы избежать переполнения.
if (weight < TINY_MAX_ELEMS && tiny.count + weight <= TINY_MAX_ELEMS)
{
for (size_t i = 0; i < weight; ++i)
tiny.insert(x);
}
else
{
if (unlikely(tiny.count <= TINY_MAX_ELEMS))
toLarge();
large->insertWeighted(x, weight);
}
}
void merge(const QuantileTiming & rhs)
{
if (tiny.count + rhs.tiny.count <= TINY_MAX_ELEMS)
@ -567,6 +589,66 @@ public:
};
/** То же самое, но с двумя аргументами. Второй аргумент - "вес" (целое число) - сколько раз учитывать значение.
*/
template <typename ArgumentFieldType, typename WeightFieldType>
class AggregateFunctionQuantileTimingWeighted final : public IAggregateFunctionHelper<QuantileTiming>
{
private:
double level;
public:
AggregateFunctionQuantileTimingWeighted(double level_ = 0.5) : level(level_) {}
String getName() const { return "quantileTimingWeighted"; }
DataTypePtr getReturnType() const
{
return new DataTypeFloat32;
}
void setArguments(const DataTypes & arguments)
{
}
void setParameters(const Array & params)
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
level = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
{
this->data(place).insertWeighted(
static_cast<const ColumnVector<ArgumentFieldType> &>(*columns[0]).getData()[row_num],
static_cast<const ColumnVector<WeightFieldType> &>(*columns[1]).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
{
this->data(place).serialize(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
{
this->data(place).deserializeMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
{
static_cast<ColumnFloat32 &>(to).getData().push_back(this->data(place).getFloat(level));
}
};
/** То же самое, но позволяет вычислить сразу несколько квантилей.
* Для этого, принимает в качестве параметров несколько уровней. Пример: quantilesTiming(0.5, 0.8, 0.9, 0.95)(ConnectTiming).
* Возвращает массив результатов.
@ -639,4 +721,75 @@ public:
}
};
template <typename ArgumentFieldType, typename WeightFieldType>
class AggregateFunctionQuantilesTimingWeighted final : public IAggregateFunctionHelper<QuantileTiming>
{
private:
typedef std::vector<double> Levels;
Levels levels;
public:
String getName() const { return "quantilesTimingWeighted"; }
DataTypePtr getReturnType() const
{
return new DataTypeArray(new DataTypeFloat32);
}
void setArguments(const DataTypes & arguments)
{
}
void setParameters(const Array & params)
{
if (params.empty())
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
size_t size = params.size();
levels.resize(size);
for (size_t i = 0; i < size; ++i)
levels[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[i]);
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
{
this->data(place).insertWeighted(
static_cast<const ColumnVector<ArgumentFieldType> &>(*columns[0]).getData()[row_num],
static_cast<const ColumnVector<WeightFieldType> &>(*columns[1]).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
{
this->data(place).serialize(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
{
this->data(place).deserializeMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
{
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
size_t size = levels.size();
offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size);
typename ColumnFloat32::Container_t & data_to = static_cast<ColumnFloat32 &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
this->data(place).getManyFloat(&levels[0], size, &data_to[old_size]);
}
};
}

View File

@ -3,41 +3,31 @@
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/AggregateFunctions/IAggregateFunction.h>
#include <DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h>
namespace DB
{
struct AggregateFunctionArgMinTraits
{
static bool better(const Field & lhs, const Field & rhs) { return lhs < rhs; }
static String name() { return "argMin"; }
};
struct AggregateFunctionArgMaxTraits
{
static bool better(const Field & lhs, const Field & rhs) { return lhs > rhs; }
static String name() { return "argMax"; }
};
/// Возможные значения параметров шаблонов см. в AggregateFunctionsMinMaxAny.h
template <typename ResultData, typename ValueData>
struct AggregateFunctionsArgMinMaxData
{
Field result; // аргумент, при котором достигается минимальное/максимальное значение value.
Field value; // значение, для которого считается минимум/максимум.
ResultData result; // аргумент, при котором достигается минимальное/максимальное значение value.
ValueData value; // значение, для которого считается минимум/максимум.
};
/// Возвращает первое попавшееся значение arg для минимального/максимального value. Пример: argMax(arg, value).
template <typename Traits>
class AggregateFunctionsArgMinMax final : public IAggregateFunctionHelper<AggregateFunctionsArgMinMaxData>
template <typename Data>
class AggregateFunctionsArgMinMax final : public IAggregateFunctionHelper<Data>
{
private:
DataTypePtr type_res;
DataTypePtr type_val;
public:
String getName() const { return Traits::name(); }
String getName() const { return (0 == strcmp(decltype(Data::value)::name(), "min")) ? "argMin" : "argMax"; }
DataTypePtr getReturnType() const
{
@ -55,105 +45,37 @@ public:
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
{
Field result;
Field value;
columns[0]->get(row_num, result);
columns[1]->get(row_num, value);
Data & d = data(place);
if (!d.value.isNull())
{
if (Traits::better(value, d.value))
{
d.result = result;
d.value = value;
}
}
else
{
d.result = result;
d.value = value;
}
if (this->data(place).value.changeIfBetter(*columns[1], row_num))
this->data(place).result.change(*columns[0], row_num);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
{
Data & d = data(place);
const Data & d_rhs = data(rhs);
if (!d.value.isNull())
{
if (Traits::better(d_rhs.value, d.value))
{
d.result = d_rhs.result;
d.value = d_rhs.value;
}
}
else
{
d.result = d_rhs.result;
d.value = d_rhs.value;
}
if (this->data(place).value.changeIfBetter(this->data(rhs).value))
this->data(place).result.change(this->data(rhs).result);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
{
const Data & d = data(place);
if (unlikely(d.result.isNull()))
{
writeBinary(false, buf);
}
else
{
writeBinary(true, buf);
type_res->serializeBinary(d.result, buf);
type_val->serializeBinary(d.value, buf);
}
this->data(place).result.write(buf, *type_res.get());
this->data(place).value.write(buf, *type_val.get());
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
{
Data & d = data(place);
Data rhs; /// Для строчек не очень оптимально, так как может делаться одна лишняя аллокация.
bool is_not_null = false;
readBinary(is_not_null, buf);
rhs.result.read(buf, *type_res.get());
rhs.value.read(buf, *type_val.get());
if (is_not_null)
{
if (!d.value.isNull())
{
Field result_;
Field value_;
type_res->deserializeBinary(result_, buf);
type_val->deserializeBinary(value_, buf);
if (Traits::better(value_, d.value))
{
d.result = result_;
d.value = value_;
}
}
else
{
type_res->deserializeBinary(d.result, buf);
type_val->deserializeBinary(d.value, buf);
}
}
if (this->data(place).value.changeIfBetter(rhs.value))
this->data(place).result.change(rhs.result);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
{
if (unlikely(data(place).value.isNull()))
to.insertDefault();
else
to.insert(data(place).result);
this->data(place).result.insertResultInto(to);
}
};
typedef AggregateFunctionsArgMinMax<AggregateFunctionArgMinTraits> AggregateFunctionArgMin;
typedef AggregateFunctionsArgMinMax<AggregateFunctionArgMaxTraits> AggregateFunctionArgMax;
}

View File

@ -67,40 +67,70 @@ struct SingleValueDataFixed
value = to.value;
}
void changeFirstTime(const IColumn & column, size_t row_num)
bool changeFirstTime(const IColumn & column, size_t row_num)
{
if (!has())
{
change(column, row_num);
return true;
}
else
return false;
}
void changeFirstTime(const Self & to)
bool changeFirstTime(const Self & to)
{
if (!has())
{
change(to);
return true;
}
else
return false;
}
void changeIfLess(const IColumn & column, size_t row_num)
bool changeIfLess(const IColumn & column, size_t row_num)
{
if (!has() || static_cast<const ColumnVector<T> &>(column).getData()[row_num] < value)
{
change(column, row_num);
return true;
}
else
return false;
}
void changeIfLess(const Self & to)
bool changeIfLess(const Self & to)
{
if (to.has() && (!has() || to.value < value))
{
change(to);
return true;
}
else
return false;
}
void changeIfGreater(const IColumn & column, size_t row_num)
bool changeIfGreater(const IColumn & column, size_t row_num)
{
if (!has() || static_cast<const ColumnVector<T> &>(column).getData()[row_num] > value)
{
change(column, row_num);
return true;
}
else
return false;
}
void changeIfGreater(const Self & to)
bool changeIfGreater(const Self & to)
{
if (to.has() && (!has() || to.value > value))
{
change(to);
return true;
}
else
return false;
}
};
@ -238,40 +268,70 @@ struct __attribute__((__packed__)) SingleValueDataString
changeImpl(to.getStringRef());
}
void changeFirstTime(const IColumn & column, size_t row_num)
bool changeFirstTime(const IColumn & column, size_t row_num)
{
if (!has())
{
change(column, row_num);
return true;
}
else
return false;
}
void changeFirstTime(const Self & to)
bool changeFirstTime(const Self & to)
{
if (!has())
{
change(to);
return true;
}
else
return false;
}
void changeIfLess(const IColumn & column, size_t row_num)
bool changeIfLess(const IColumn & column, size_t row_num)
{
if (!has() || static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num) < getStringRef())
{
change(column, row_num);
return true;
}
else
return false;
}
void changeIfLess(const Self & to)
bool changeIfLess(const Self & to)
{
if (to.has() && (!has() || to.getStringRef() < getStringRef()))
{
change(to);
return true;
}
else
return false;
}
void changeIfGreater(const IColumn & column, size_t row_num)
bool changeIfGreater(const IColumn & column, size_t row_num)
{
if (!has() || static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num) > getStringRef())
{
change(column, row_num);
return true;
}
else
return false;
}
void changeIfGreater(const Self & to)
bool changeIfGreater(const Self & to)
{
if (to.has() && (!has() || to.getStringRef() > getStringRef()))
{
change(to);
return true;
}
else
return false;
}
};
@ -326,54 +386,90 @@ struct SingleValueDataGeneric
value = to.value;
}
void changeFirstTime(const IColumn & column, size_t row_num)
bool changeFirstTime(const IColumn & column, size_t row_num)
{
if (!has())
{
change(column, row_num);
return true;
}
else
return false;
}
void changeFirstTime(const Self & to)
bool changeFirstTime(const Self & to)
{
if (!has())
{
change(to);
return true;
}
else
return false;
}
void changeIfLess(const IColumn & column, size_t row_num)
bool changeIfLess(const IColumn & column, size_t row_num)
{
if (!has())
{
change(column, row_num);
return true;
}
else
{
Field new_value;
column.get(row_num, new_value);
if (new_value < value)
{
value = new_value;
return true;
}
else
return false;
}
}
void changeIfLess(const Self & to)
bool changeIfLess(const Self & to)
{
if (to.has() && (!has() || to.value < value))
{
change(to);
return true;
}
else
return false;
}
void changeIfGreater(const IColumn & column, size_t row_num)
bool changeIfGreater(const IColumn & column, size_t row_num)
{
if (!has())
{
change(column, row_num);
return true;
}
else
{
Field new_value;
column.get(row_num, new_value);
if (new_value > value)
{
value = new_value;
return true;
}
else
return false;
}
}
void changeIfGreater(const Self & to)
bool changeIfGreater(const Self & to)
{
if (to.has() && (!has() || to.value > value))
{
change(to);
return true;
}
else
return false;
}
};
@ -388,8 +484,8 @@ struct AggregateFunctionMinData : Data
{
typedef AggregateFunctionMinData<Data> Self;
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeIfLess(column, row_num); }
void changeIfBetter(const Self & to) { this->changeIfLess(to); }
bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeIfLess(column, row_num); }
bool changeIfBetter(const Self & to) { return this->changeIfLess(to); }
static const char * name() { return "min"; }
};
@ -399,8 +495,8 @@ struct AggregateFunctionMaxData : Data
{
typedef AggregateFunctionMaxData<Data> Self;
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeIfGreater(column, row_num); }
void changeIfBetter(const Self & to) { this->changeIfGreater(to); }
bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeIfGreater(column, row_num); }
bool changeIfBetter(const Self & to) { return this->changeIfGreater(to); }
static const char * name() { return "max"; }
};
@ -410,8 +506,8 @@ struct AggregateFunctionAnyData : Data
{
typedef AggregateFunctionAnyData<Data> Self;
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeFirstTime(column, row_num); }
void changeIfBetter(const Self & to) { this->changeFirstTime(to); }
bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeFirstTime(column, row_num); }
bool changeIfBetter(const Self & to) { return this->changeFirstTime(to); }
static const char * name() { return "any"; }
};
@ -421,8 +517,8 @@ struct AggregateFunctionAnyLastData : Data
{
typedef AggregateFunctionAnyLastData<Data> Self;
void changeIfBetter(const IColumn & column, size_t row_num) { this->change(column, row_num); }
void changeIfBetter(const Self & to) { this->change(to); }
bool changeIfBetter(const IColumn & column, size_t row_num) { this->change(column, row_num); return true; }
bool changeIfBetter(const Self & to) { this->change(to); return true; }
static const char * name() { return "anyLast"; }
};

View File

@ -1,7 +1,6 @@
#pragma once
#include <DB/Columns/IColumn.h>
#include <DB/Interpreters/Set.h>
namespace DB

View File

@ -185,6 +185,24 @@ public:
}
/// То же самое, но вернуть false, если переполнено.
bool ALWAYS_INLINE tryEmplace(Key x, iterator & it, bool & inserted)
{
Cell * res = findCell(x);
it = iteratorTo(res);
inserted = res == buf + m_size;
if (inserted)
{
if (res == buf + capacity)
return false;
new(res) Cell(x, *this);
++m_size;
}
return true;
}
/// Скопировать ячейку из другой хэш-таблицы. Предполагается, что такого ключа в таблице ещё не было.
void ALWAYS_INLINE insertUnique(const Cell * cell)
{
@ -192,6 +210,12 @@ public:
++m_size;
}
void ALWAYS_INLINE insertUnique(Key x)
{
new(&buf[m_size]) Cell(x, *this);
++m_size;
}
iterator ALWAYS_INLINE find(Key x) { return iteratorTo(findCell(x)); }
const_iterator ALWAYS_INLINE find(Key x) const { return iteratorTo(findCell(x)); }

View File

@ -276,6 +276,7 @@ namespace ErrorCodes
MISMATCH_REPLICAS_DATA_SOURCES,
STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS,
CPUID_ERROR,
INFINITE_LOOP,
AIO_SUBMIT_ERROR,
AIO_COMPLETION_ERROR,

View File

@ -25,7 +25,7 @@ struct StringRef
std::string toString() const { return std::string(data, size); }
operator std::string() const { return toString(); }
explicit operator std::string() const { return toString(); }
};
typedef std::vector<StringRef> StringRefs;

View File

@ -2,8 +2,6 @@
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Interpreters/ExpressionAnalyzer.h>
#include <DB/Interpreters/Set.h>
#include <DB/Interpreters/Join.h>
namespace DB

View File

@ -52,15 +52,25 @@ public:
const DictionaryLifetime & getLifetime() const override { return dict_lifetime; }
bool hasHierarchy() const override { return false; }
bool hasHierarchy() const override { return hierarchical_attribute; }
id_t toParent(const id_t id) const override { return 0; }
id_t toParent(const id_t id) const override
{
PODArray<UInt64> ids{1, id};
PODArray<UInt64> out{1};
getItems<UInt64>(*hierarchical_attribute, ids, out);
return out.front();
}
void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const override
{
getItems<UInt64>(*hierarchical_attribute, ids, out);
}
#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
const auto idx = getAttributeIndex(attribute_name);\
const auto & attribute = attributes[idx];\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
@ -69,7 +79,7 @@ public:
\
PODArray<UInt64> ids{1, id};\
PODArray<TYPE> out{1};\
getItems<TYPE>(idx, ids, out);\
getItems<TYPE>(attribute, ids, out);\
return out.front();\
}
DECLARE_INDIVIDUAL_GETTER(UInt8, uint8)
@ -85,8 +95,7 @@ public:
#undef DECLARE_INDIVIDUAL_GETTER
String getString(const std::string & attribute_name, const id_t id) const override
{
const auto idx = getAttributeIndex(attribute_name);
const auto & attribute = attributes[idx];
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
@ -95,23 +104,22 @@ public:
PODArray<UInt64> ids{1, id};
ColumnString out;
getItems(idx, ids, &out);
getItems(attribute, ids, &out);
return out.getDataAt(0);
}
return String{out.getDataAt(0)};
};
#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
{\
const auto idx = getAttributeIndex(attribute_name);\
const auto & attribute = attributes[idx];\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(idx, ids, out);\
getItems<TYPE>(attribute, ids, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8, uint8)
DECLARE_MULTIPLE_GETTER(UInt16, uint16)
@ -126,15 +134,14 @@ public:
#undef DECLARE_MULTIPLE_GETTER
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const override
{
const auto idx = getAttributeIndex(attribute_name);
const auto & attribute = attributes[idx];
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
getItems(idx, ids, out);
getItems(attribute, ids, out);
}
private:
@ -176,7 +183,15 @@ private:
attribute.null_value));
if (attribute.hierarchical)
{
hierarchical_attribute = &attributes.back();
if (hierarchical_attribute->type != AttributeType::uint64)
throw Exception{
"Hierarchical attribute must be UInt64.",
ErrorCodes::TYPE_MISMATCH
};
}
}
}
@ -235,20 +250,16 @@ private:
return attr;
}
static bool hasTimeExpired(const std::chrono::system_clock::time_point & time_point)
{
return std::chrono::system_clock::now() >= time_point;
}
template <typename T>
void getItems(const std::size_t attribute_idx, const PODArray<id_t> & ids, PODArray<T> & out) const
void getItems(attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out) const
{
HashMap<id_t, std::vector<std::size_t>> outdated_ids;
auto & attribute = attributes[attribute_idx];
auto & attribute_array = std::get<std::unique_ptr<T[]>>(attribute.arrays);
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto i : ext::range(0, ids.size()))
{
@ -262,7 +273,7 @@ private:
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
if (cell.id != id || hasTimeExpired(cell.expires_at))
if (cell.id != id || cell.expires_at < now)
{
out[i] = std::get<T>(attribute.null_values);
outdated_ids[id].push_back(i);
@ -289,12 +300,11 @@ private:
});
}
void getItems(const std::size_t attribute_idx, const PODArray<id_t> & ids, ColumnString * out) const
void getItems(attribute_t & attribute, const PODArray<id_t> & ids, ColumnString * out) const
{
/// save on some allocations
out->getOffsets().reserve(ids.size());
auto & attribute = attributes[attribute_idx];
auto & attribute_array = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays);
auto found_outdated_values = false;
@ -303,6 +313,7 @@ private:
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, discard on fail
for (const auto i : ext::range(0, ids.size()))
{
@ -317,7 +328,7 @@ private:
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
if (cell.id != id || hasTimeExpired(cell.expires_at))
if (cell.id != id || cell.expires_at < now)
{
found_outdated_values = true;
break;
@ -347,6 +358,7 @@ private:
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
for (const auto i : ext::range(0, ids.size()))
{
const auto id = ids[i];
@ -359,14 +371,14 @@ private:
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
if (cell.id != id || hasTimeExpired(cell.expires_at))
if (cell.id != id || cell.expires_at < now)
outdated_ids[id] += 1;
else
{
const auto string_ref = attribute_array[cell_idx];
map[id] = string_ref;
map[id] = String{string_ref};
total_length += string_ref.size + 1;
};
}
}
}
@ -380,8 +392,8 @@ private:
update(required_ids, [&] (const auto id, const auto cell_idx) {
const auto attribute_value = attribute_array[cell_idx];
map[id] = attribute_value;
total_length += attribute_value.size + 1;
map[id] = String{attribute_value};
total_length += (attribute_value.size + 1) * outdated_ids[id];
});
}
@ -390,7 +402,7 @@ private:
for (const auto id : ids)
{
const auto it = map.find(id);
const auto string = it != map.end() ? it->second : std::get<String>(attributes[attribute_idx].null_values);
const auto string = it != map.end() ? it->second : std::get<String>(attribute.null_values);
out->insertData(string.data(), string.size());
}
}
@ -414,6 +426,11 @@ private:
const auto & ids = id_column->getData();
/// cache column pointers
std::vector<const IColumn *> column_ptrs(attributes.size());
for (const auto i : ext::range(0, attributes.size()))
column_ptrs[i] = block.getByPosition(i + 1).column.get();
for (const auto i : ext::range(0, ids.size()))
{
const auto id = ids[i];
@ -422,7 +439,7 @@ private:
for (const auto attribute_idx : ext::range(0, attributes.size()))
{
const auto & attribute_column = *block.getByPosition(attribute_idx + 1).column;
const auto & attribute_column = *column_ptrs[attribute_idx];
auto & attribute = attributes[attribute_idx];
setAttributeValue(attribute, cell_idx, attribute_column[i]);
@ -486,7 +503,7 @@ private:
}
}
std::size_t getAttributeIndex(const std::string & attribute_name) const
attribute_t & getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
@ -495,7 +512,7 @@ private:
ErrorCodes::BAD_ARGUMENTS
};
return it->second;
return attributes[it->second];
}
static std::size_t round_up_to_power_of_two(std::size_t n)
@ -529,7 +546,7 @@ private:
std::map<std::string, std::size_t> attribute_index_by_name;
mutable std::vector<attribute_t> attributes;
mutable std::vector<cell_metadata_t> cells;
const attribute_t * hierarchical_attribute = nullptr;
attribute_t * hierarchical_attribute = nullptr;
mutable std::mt19937_64 rnd_engine{getSeed()};
};

View File

@ -11,7 +11,7 @@
namespace DB
{
const auto max_connections = 1;
const auto max_connections = 16;
/** Allows loading dictionaries from local or remote ClickHouse instance
* @todo use ConnectionPoolWithFailover
@ -75,56 +75,70 @@ public:
DictionarySourcePtr clone() const override { return std::make_unique<ClickHouseDictionarySource>(*this); }
private:
/// @todo escape table and column names
static std::string composeLoadAllQuery(const Block & block, const std::string & table)
{
std::string query{"SELECT "};
std::string query;
auto first = true;
for (const auto idx : ext::range(0, block.columns()))
{
if (!first)
query += ", ";
WriteBufferFromString out{query};
writeString("SELECT ", out);
query += block.getByPosition(idx).name;
first = false;
auto first = true;
for (const auto idx : ext::range(0, block.columns()))
{
if (!first)
writeString(", ", out);
writeString(block.getByPosition(idx).name, out);
first = false;
}
writeString(" FROM ", out);
writeProbablyBackQuotedString(table, out);
writeChar(';', out);
}
query += " FROM " + table + ';';
return query;
}
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> ids)
{
std::string query{"SELECT "};
std::string query;
auto first = true;
for (const auto idx : ext::range(0, sample_block.columns()))
{
if (!first)
query += ", ";
WriteBufferFromString out{query};
writeString("SELECT ", out);
first = false;
query += sample_block.getByPosition(idx).name;
auto first = true;
for (const auto idx : ext::range(0, sample_block.columns()))
{
if (!first)
writeString(", ", out);
writeString(sample_block.getByPosition(idx).name, out);
first = false;
}
const auto & id_column_name = sample_block.getByPosition(0).name;
writeString(" FROM ", out);
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
writeProbablyBackQuotedString(id_column_name, out);
writeString(" IN (", out);
first = true;
for (const auto id : ids)
{
if (!first)
writeString(", ", out);
first = false;
writeString(toString(id), out);
}
writeString(");", out);
}
const auto & id_column_name = sample_block.getByPosition(0).name;
query += " FROM " + table + " WHERE " + id_column_name + " IN (";
first = true;
for (const auto id : ids)
{
if (!first)
query += ',';
first = false;
query += toString(id);
}
query += ");";
return query;
}

View File

@ -6,6 +6,7 @@
#include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp>
#include <vector>
#include <tuple>
namespace DB
{
@ -46,42 +47,29 @@ public:
id_t toParent(const id_t id) const override
{
const auto attr = hierarchical_attribute;
const auto & array = *std::get<std::unique_ptr<PODArray<UInt64>>>(attr->arrays);
switch (hierarchical_attribute->type)
{
case AttributeType::uint8: return id < attr->uint8_array->size() ? (*attr->uint8_array)[id] : attr->uint8_null_value;
case AttributeType::uint16: return id < attr->uint16_array->size() ? (*attr->uint16_array)[id] : attr->uint16_null_value;
case AttributeType::uint32: return id < attr->uint32_array->size() ? (*attr->uint32_array)[id] : attr->uint32_null_value;
case AttributeType::uint64: return id < attr->uint64_array->size() ? (*attr->uint64_array)[id] : attr->uint64_null_value;
case AttributeType::int8: return id < attr->int8_array->size() ? (*attr->int8_array)[id] : attr->int8_null_value;
case AttributeType::int16: return id < attr->int16_array->size() ? (*attr->int16_array)[id] : attr->int16_null_value;
case AttributeType::int32: return id < attr->int32_array->size() ? (*attr->int32_array)[id] : attr->int32_null_value;
case AttributeType::int64: return id < attr->int64_array->size() ? (*attr->int64_array)[id] : attr->int64_null_value;
case AttributeType::float32:
case AttributeType::float64:
case AttributeType::string:
break;
}
return id < array.size() ? array[id] : std::get<UInt64>(attr->null_values);
}
throw Exception{
"Hierarchical attribute has non-integer type " + toString(hierarchical_attribute->type),
ErrorCodes::TYPE_MISMATCH
};
void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const override
{
getItems<UInt64>(*hierarchical_attribute, ids, out);
}
#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
const auto idx = getAttributeIndex(attribute_name);\
const auto & attribute = attributes[idx];\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
if (id < attribute.LC_TYPE##_array->size())\
return (*attribute.LC_TYPE##_array)[id];\
return attribute.LC_TYPE##_null_value;\
\
const auto & array = *std::get<std::unique_ptr<PODArray<TYPE>>>(attribute.arrays);\
\
return id < array.size() ? array[id] : std::get<TYPE>(attribute.null_values);\
}
DECLARE_INDIVIDUAL_GETTER(UInt8, uint8)
DECLARE_INDIVIDUAL_GETTER(UInt16, uint16)
@ -93,28 +81,32 @@ public:
DECLARE_INDIVIDUAL_GETTER(Int64, int64)
DECLARE_INDIVIDUAL_GETTER(Float32, float32)
DECLARE_INDIVIDUAL_GETTER(Float64, float64)
DECLARE_INDIVIDUAL_GETTER(String, string)
#undef DECLARE_INDIVIDUAL_GETTER
String getString(const std::string & attribute_name, const id_t id) const override
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
const auto & array = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);
return id < array.size() ? String{array[id]} : std::get<String>(attribute.null_values);
}
#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
{\
const auto idx = getAttributeIndex(attribute_name);\
const auto & attribute = attributes[idx];\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
const auto & attr = *attribute.LC_TYPE##_array;\
const auto null_value = attribute.LC_TYPE##_null_value;\
\
for (const auto i : ext::range(0, ids.size()))\
{\
const auto id = ids[i];\
out[i] = id < attr.size() ? attr[id] : null_value;\
}\
getItems<TYPE>(attribute, ids, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8, uint8)
DECLARE_MULTIPLE_GETTER(UInt16, uint16)
@ -129,16 +121,15 @@ public:
#undef DECLARE_MULTIPLE_GETTER
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const override
{
const auto idx = getAttributeIndex(attribute_name);
const auto & attribute = attributes[idx];
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
const auto & attr = *attribute.string_array;
const auto null_value = attribute.string_null_value;
const auto & attr = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);
const auto & null_value = std::get<String>(attribute.null_values);
for (const auto i : ext::range(0, ids.size()))
{
@ -149,38 +140,32 @@ public:
}
private:
struct attribute_t
struct attribute_t final
{
AttributeType type;
UInt8 uint8_null_value;
UInt16 uint16_null_value;
UInt32 uint32_null_value;
UInt64 uint64_null_value;
Int8 int8_null_value;
Int16 int16_null_value;
Int32 int32_null_value;
Int64 int64_null_value;
Float32 float32_null_value;
Float64 float64_null_value;
String string_null_value;
std::unique_ptr<PODArray<UInt8>> uint8_array;
std::unique_ptr<PODArray<UInt16>> uint16_array;
std::unique_ptr<PODArray<UInt32>> uint32_array;
std::unique_ptr<PODArray<UInt64>> uint64_array;
std::unique_ptr<PODArray<Int8>> int8_array;
std::unique_ptr<PODArray<Int16>> int16_array;
std::unique_ptr<PODArray<Int32>> int32_array;
std::unique_ptr<PODArray<Int64>> int64_array;
std::unique_ptr<PODArray<Float32>> float32_array;
std::unique_ptr<PODArray<Float64>> float64_array;
std::tuple<UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<std::unique_ptr<PODArray<UInt8>>,
std::unique_ptr<PODArray<UInt16>>,
std::unique_ptr<PODArray<UInt32>>,
std::unique_ptr<PODArray<UInt64>>,
std::unique_ptr<PODArray<Int8>>,
std::unique_ptr<PODArray<Int16>>,
std::unique_ptr<PODArray<Int32>>,
std::unique_ptr<PODArray<Int64>>,
std::unique_ptr<PODArray<Float32>>,
std::unique_ptr<PODArray<Float64>>,
std::unique_ptr<PODArray<StringRef>>> arrays;
std::unique_ptr<Arena> string_arena;
std::unique_ptr<PODArray<StringRef>> string_array;
};
void createAttributes()
{
const auto size = dict_struct.attributes.size();
attributes.reserve(size);
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
@ -188,7 +173,15 @@ private:
attribute.null_value));
if (attribute.hierarchical)
{
hierarchical_attribute = &attributes.back();
if (hierarchical_attribute->type != AttributeType::uint64)
throw Exception{
"Hierarchical attribute must be UInt64.",
ErrorCodes::TYPE_MISMATCH
};
}
}
}
@ -214,73 +207,65 @@ private:
stream->readSuffix();
}
template <typename T>
void createAttributeImpl(attribute_t & attribute, const std::string & null_value)
{
const auto & null_value_ref = std::get<T>(attribute.null_values) = DB::parse<T>(null_value);
std::get<std::unique_ptr<PODArray<T>>>(attribute.arrays) =
std::make_unique<PODArray<T>>(initial_array_size, null_value_ref);
}
attribute_t createAttributeWithType(const AttributeType type, const std::string & null_value)
{
attribute_t attr{type};
switch (type)
{
case AttributeType::uint8:
attr.uint8_null_value = DB::parse<UInt8>(null_value);
attr.uint8_array.reset(new PODArray<UInt8>);
attr.uint8_array->resize_fill(initial_array_size, attr.uint8_null_value);
break;
case AttributeType::uint16:
attr.uint16_null_value = DB::parse<UInt16>(null_value);
attr.uint16_array.reset(new PODArray<UInt16>);
attr.uint16_array->resize_fill(initial_array_size, attr.uint16_null_value);
break;
case AttributeType::uint32:
attr.uint32_null_value = DB::parse<UInt32>(null_value);
attr.uint32_array.reset(new PODArray<UInt32>);
attr.uint32_array->resize_fill(initial_array_size, attr.uint32_null_value);
break;
case AttributeType::uint64:
attr.uint64_null_value = DB::parse<UInt64>(null_value);
attr.uint64_array.reset(new PODArray<UInt64>);
attr.uint64_array->resize_fill(initial_array_size, attr.uint64_null_value);
break;
case AttributeType::int8:
attr.int8_null_value = DB::parse<Int8>(null_value);
attr.int8_array.reset(new PODArray<Int8>);
attr.int8_array->resize_fill(initial_array_size, attr.int8_null_value);
break;
case AttributeType::int16:
attr.int16_null_value = DB::parse<Int16>(null_value);
attr.int16_array.reset(new PODArray<Int16>);
attr.int16_array->resize_fill(initial_array_size, attr.int16_null_value);
break;
case AttributeType::int32:
attr.int32_null_value = DB::parse<Int32>(null_value);
attr.int32_array.reset(new PODArray<Int32>);
attr.int32_array->resize_fill(initial_array_size, attr.int32_null_value);
break;
case AttributeType::int64:
attr.int64_null_value = DB::parse<Int64>(null_value);
attr.int64_array.reset(new PODArray<Int64>);
attr.int64_array->resize_fill(initial_array_size, attr.int64_null_value);
break;
case AttributeType::float32:
attr.float32_null_value = DB::parse<Float32>(null_value);
attr.float32_array.reset(new PODArray<Float32>);
attr.float32_array->resize_fill(initial_array_size, attr.float32_null_value);
break;
case AttributeType::float64:
attr.float64_null_value = DB::parse<Float64>(null_value);
attr.float64_array.reset(new PODArray<Float64>);
attr.float64_array->resize_fill(initial_array_size, attr.float64_null_value);
break;
case AttributeType::uint8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeType::uint16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeType::uint32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeType::uint64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeType::int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeType::int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeType::int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeType::int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeType::float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeType::float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeType::string:
attr.string_null_value = null_value;
attr.string_arena.reset(new Arena);
attr.string_array.reset(new PODArray<StringRef>);
attr.string_array->resize_fill(initial_array_size, attr.string_null_value);
{
const auto & null_value_ref = std::get<String>(attr.null_values) = DB::parse<String>(null_value);
std::get<std::unique_ptr<PODArray<StringRef>>>(attr.arrays) =
std::make_unique<PODArray<StringRef>>(initial_array_size, null_value_ref);
attr.string_arena = std::make_unique<Arena>();
break;
}
}
};
return attr;
}
template <typename T>
void getItems(const attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out) const
{
const auto & attr = *std::get<std::unique_ptr<PODArray<T>>>(attribute.arrays);
const auto null_value = std::get<T>(attribute.null_values);
for (const auto i : ext::range(0, ids.size()))
{
const auto id = ids[i];
out[i] = id < attr.size() ? attr[id] : null_value;
}
}
template <typename T>
void setAttributeValueImpl(attribute_t & attribute, const id_t id, const T value)
{
auto & array = *std::get<std::unique_ptr<PODArray<T>>>(attribute.arrays);
if (id >= array.size())
array.resize_fill(id, std::get<T>(attribute.null_values));
array[id] = value;
}
void setAttributeValue(attribute_t & attribute, const id_t id, const Field & value)
{
if (id >= max_array_size)
@ -291,89 +276,30 @@ private:
switch (attribute.type)
{
case AttributeType::uint8:
{
if (id >= attribute.uint8_array->size())
attribute.uint8_array->resize_fill(id, attribute.uint8_null_value);
(*attribute.uint8_array)[id] = value.get<UInt64>();
break;
}
case AttributeType::uint16:
{
if (id >= attribute.uint16_array->size())
attribute.uint16_array->resize_fill(id, attribute.uint16_null_value);
(*attribute.uint16_array)[id] = value.get<UInt64>();
break;
}
case AttributeType::uint32:
{
if (id >= attribute.uint32_array->size())
attribute.uint32_array->resize_fill(id, attribute.uint32_null_value);
(*attribute.uint32_array)[id] = value.get<UInt64>();
break;
}
case AttributeType::uint64:
{
if (id >= attribute.uint64_array->size())
attribute.uint64_array->resize_fill(id, attribute.uint64_null_value);
(*attribute.uint64_array)[id] = value.get<UInt64>();
break;
}
case AttributeType::int8:
{
if (id >= attribute.int8_array->size())
attribute.int8_array->resize_fill(id, attribute.int8_null_value);
(*attribute.int8_array)[id] = value.get<Int64>();
break;
}
case AttributeType::int16:
{
if (id >= attribute.int16_array->size())
attribute.int16_array->resize_fill(id, attribute.int16_null_value);
(*attribute.int16_array)[id] = value.get<Int64>();
break;
}
case AttributeType::int32:
{
if (id >= attribute.int32_array->size())
attribute.int32_array->resize_fill(id, attribute.int32_null_value);
(*attribute.int32_array)[id] = value.get<Int64>();
break;
}
case AttributeType::int64:
{
if (id >= attribute.int64_array->size())
attribute.int64_array->resize_fill(id, attribute.int64_null_value);
(*attribute.int64_array)[id] = value.get<Int64>();
break;
}
case AttributeType::float32:
{
if (id >= attribute.float32_array->size())
attribute.float32_array->resize_fill(id, attribute.float32_null_value);
(*attribute.float32_array)[id] = value.get<Float64>();
break;
}
case AttributeType::float64:
{
if (id >= attribute.float64_array->size())
attribute.float64_array->resize_fill(id, attribute.float64_null_value);
(*attribute.float64_array)[id] = value.get<Float64>();
break;
}
case AttributeType::uint8: setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); break;
case AttributeType::uint16: setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); break;
case AttributeType::uint32: setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>()); break;
case AttributeType::uint64: setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>()); break;
case AttributeType::int8: setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>()); break;
case AttributeType::int16: setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>()); break;
case AttributeType::int32: setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>()); break;
case AttributeType::int64: setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>()); break;
case AttributeType::float32: setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>()); break;
case AttributeType::float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeType::string:
{
if (id >= attribute.string_array->size())
attribute.string_array->resize_fill(id, attribute.string_null_value);
auto & array = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);
if (id >= array.size())
array.resize_fill(id, std::get<String>(attribute.null_values));
const auto & string = value.get<String>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
(*attribute.string_array)[id] = StringRef{string_in_arena, string.size()};
array[id] = StringRef{string_in_arena, string.size()};
break;
}
}
}
std::size_t getAttributeIndex(const std::string & attribute_name) const
const attribute_t & getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
@ -382,7 +308,7 @@ private:
ErrorCodes::BAD_ARGUMENTS
};
return it->second;
return attributes[it->second];
}
const std::string name;

View File

@ -7,6 +7,7 @@
#include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp>
#include <memory>
#include <tuple>
namespace DB
{
@ -44,77 +45,31 @@ public:
id_t toParent(const id_t id) const override
{
const auto attr = hierarchical_attribute;
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, UInt64>>>(attr->maps);
const auto it = map.find(id);
switch (hierarchical_attribute->type)
{
case AttributeType::uint8:
{
const auto it = attr->uint8_map->find(id);
return it != attr->uint8_map->end() ? it->second : attr->uint8_null_value;
}
case AttributeType::uint16:
{
const auto it = attr->uint16_map->find(id);
return it != attr->uint16_map->end() ? it->second : attr->uint16_null_value;
}
case AttributeType::uint32:
{
const auto it = attr->uint32_map->find(id);
return it != attr->uint32_map->end() ? it->second : attr->uint32_null_value;
}
case AttributeType::uint64:
{
const auto it = attr->uint64_map->find(id);
return it != attr->uint64_map->end() ? it->second : attr->uint64_null_value;
}
case AttributeType::int8:
{
const auto it = attr->int8_map->find(id);
return it != attr->int8_map->end() ? it->second : attr->int8_null_value;
}
case AttributeType::int16:
{
const auto it = attr->int16_map->find(id);
return it != attr->int16_map->end() ? it->second : attr->int16_null_value;
}
case AttributeType::int32:
{
const auto it = attr->int32_map->find(id);
return it != attr->int32_map->end() ? it->second : attr->int32_null_value;
}
case AttributeType::int64:
{
const auto it = attr->int64_map->find(id);
return it != attr->int64_map->end() ? it->second : attr->int64_null_value;
}
case AttributeType::float32:
case AttributeType::float64:
case AttributeType::string:
break;
};
return it != map.end() ? it->second : std::get<UInt64>(attr->null_values);
}
throw Exception{
"Hierarchical attribute has non-integer type " + toString(hierarchical_attribute->type),
ErrorCodes::TYPE_MISMATCH
};
void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const override
{
getItems<UInt64>(*hierarchical_attribute, ids, out);
}
#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
const auto idx = getAttributeIndex(attribute_name);\
const auto & attribute = attributes[idx];\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
const auto it = attribute.LC_TYPE##_map->find(id);\
if (it != attribute.LC_TYPE##_map->end())\
return it->second;\
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, TYPE>>>(attribute.maps);\
const auto it = map.find(id);\
\
return attribute.LC_TYPE##_null_value;\
return it != map.end() ? TYPE{it->second} : std::get<TYPE>(attribute.null_values);\
}
DECLARE_INDIVIDUAL_GETTER(UInt8, uint8)
DECLARE_INDIVIDUAL_GETTER(UInt16, uint16)
@ -126,28 +81,33 @@ public:
DECLARE_INDIVIDUAL_GETTER(Int64, int64)
DECLARE_INDIVIDUAL_GETTER(Float32, float32)
DECLARE_INDIVIDUAL_GETTER(Float64, float64)
DECLARE_INDIVIDUAL_GETTER(String, string)
#undef DECLARE_INDIVIDUAL_GETTER
String getString(const std::string & attribute_name, const id_t id) const override
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
const auto & map = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
const auto it = map.find(id);
return it != map.end() ? String{it->second} : std::get<String>(attribute.null_values);
}
#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
{\
const auto idx = getAttributeIndex(attribute_name);\
const auto & attribute = attributes[idx];\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
const auto & attr = *attribute.LC_TYPE##_map;\
const auto null_value = attribute.LC_TYPE##_null_value;\
\
for (const auto i : ext::range(0, ids.size()))\
{\
const auto it = attr.find(ids[i]);\
out[i] = it != attr.end() ? it->second : null_value;\
}\
getItems<TYPE>(attribute, ids, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8, uint8)
DECLARE_MULTIPLE_GETTER(UInt16, uint16)
@ -162,16 +122,15 @@ public:
#undef DECLARE_MULTIPLE_GETTER
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const override
{
const auto idx = getAttributeIndex(attribute_name);
const auto & attribute = attributes[idx];
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
const auto & attr = *attribute.string_map;
const auto null_value = attribute.string_null_value;
const auto & attr = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
const auto & null_value = std::get<String>(attribute.null_values);
for (const auto i : ext::range(0, ids.size()))
{
@ -182,38 +141,32 @@ public:
}
private:
struct attribute_t
struct attribute_t final
{
AttributeType type;
UInt8 uint8_null_value;
UInt16 uint16_null_value;
UInt32 uint32_null_value;
UInt64 uint64_null_value;
Int8 int8_null_value;
Int16 int16_null_value;
Int32 int32_null_value;
Int64 int64_null_value;
Float32 float32_null_value;
Float64 float64_null_value;
String string_null_value;
std::unique_ptr<HashMap<UInt64, UInt8>> uint8_map;
std::unique_ptr<HashMap<UInt64, UInt16>> uint16_map;
std::unique_ptr<HashMap<UInt64, UInt32>> uint32_map;
std::unique_ptr<HashMap<UInt64, UInt64>> uint64_map;
std::unique_ptr<HashMap<UInt64, Int8>> int8_map;
std::unique_ptr<HashMap<UInt64, Int16>> int16_map;
std::unique_ptr<HashMap<UInt64, Int32>> int32_map;
std::unique_ptr<HashMap<UInt64, Int64>> int64_map;
std::unique_ptr<HashMap<UInt64, Float32>> float32_map;
std::unique_ptr<HashMap<UInt64, Float64>> float64_map;
std::tuple<UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<std::unique_ptr<HashMap<UInt64, UInt8>>,
std::unique_ptr<HashMap<UInt64, UInt16>>,
std::unique_ptr<HashMap<UInt64, UInt32>>,
std::unique_ptr<HashMap<UInt64, UInt64>>,
std::unique_ptr<HashMap<UInt64, Int8>>,
std::unique_ptr<HashMap<UInt64, Int16>>,
std::unique_ptr<HashMap<UInt64, Int32>>,
std::unique_ptr<HashMap<UInt64, Int64>>,
std::unique_ptr<HashMap<UInt64, Float32>>,
std::unique_ptr<HashMap<UInt64, Float64>>,
std::unique_ptr<HashMap<UInt64, StringRef>>> maps;
std::unique_ptr<Arena> string_arena;
std::unique_ptr<HashMap<UInt64, StringRef>> string_map;
};
void createAttributes()
{
const auto size = dict_struct.attributes.size();
attributes.reserve(size);
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
@ -221,7 +174,15 @@ private:
attribute.null_value));
if (attribute.hierarchical)
{
hierarchical_attribute = &attributes.back();
if (hierarchical_attribute->type != AttributeType::uint64)
throw Exception{
"Hierarchical attribute must be UInt64.",
ErrorCodes::TYPE_MISMATCH
};
}
}
}
@ -247,127 +208,88 @@ private:
stream->readSuffix();
}
template <typename T>
void createAttributeImpl(attribute_t & attribute, const std::string & null_value)
{
std::get<T>(attribute.null_values) = DB::parse<T>(null_value);
std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps) = std::make_unique<HashMap<UInt64, T>>();
}
attribute_t createAttributeWithType(const AttributeType type, const std::string & null_value)
{
attribute_t attr{type};
switch (type)
{
case AttributeType::uint8:
attr.uint8_null_value = DB::parse<UInt8>(null_value);
attr.uint8_map.reset(new HashMap<UInt64, UInt8>);
break;
case AttributeType::uint16:
attr.uint16_null_value = DB::parse<UInt16>(null_value);
attr.uint16_map.reset(new HashMap<UInt64, UInt16>);
break;
case AttributeType::uint32:
attr.uint32_null_value = DB::parse<UInt32>(null_value);
attr.uint32_map.reset(new HashMap<UInt64, UInt32>);
break;
case AttributeType::uint64:
attr.uint64_null_value = DB::parse<UInt64>(null_value);
attr.uint64_map.reset(new HashMap<UInt64, UInt64>);
break;
case AttributeType::int8:
attr.int8_null_value = DB::parse<Int8>(null_value);
attr.int8_map.reset(new HashMap<UInt64, Int8>);
break;
case AttributeType::int16:
attr.int16_null_value = DB::parse<Int16>(null_value);
attr.int16_map.reset(new HashMap<UInt64, Int16>);
break;
case AttributeType::int32:
attr.int32_null_value = DB::parse<Int32>(null_value);
attr.int32_map.reset(new HashMap<UInt64, Int32>);
break;
case AttributeType::int64:
attr.int64_null_value = DB::parse<Int64>(null_value);
attr.int64_map.reset(new HashMap<UInt64, Int64>);
break;
case AttributeType::float32:
attr.float32_null_value = DB::parse<Float32>(null_value);
attr.float32_map.reset(new HashMap<UInt64, Float32>);
break;
case AttributeType::float64:
attr.float64_null_value = DB::parse<Float64>(null_value);
attr.float64_map.reset(new HashMap<UInt64, Float64>);
break;
case AttributeType::uint8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeType::uint16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeType::uint32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeType::uint64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeType::int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeType::int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeType::int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeType::int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeType::float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeType::float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeType::string:
attr.string_null_value = null_value;
attr.string_arena.reset(new Arena);
attr.string_map.reset(new HashMap<UInt64, StringRef>);
{
const auto & null_value_ref = std::get<String>(attr.null_values) = DB::parse<String>(null_value);
std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attr.maps) =
std::make_unique<HashMap<UInt64, StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break;
}
}
return attr;
}
template <typename T>
void getItems(const attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out) const
{
const auto & attr = *std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps);
const auto null_value = std::get<T>(attribute.null_values);
for (const auto i : ext::range(0, ids.size()))
{
const auto it = attr.find(ids[i]);
out[i] = it != attr.end() ? it->second : null_value;
}
}
template <typename T>
void setAttributeValueImpl(attribute_t & attribute, const id_t id, const T value)
{
auto & map = *std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps);
map.insert({ id, value });
}
void setAttributeValue(attribute_t & attribute, const id_t id, const Field & value)
{
switch (attribute.type)
{
case AttributeType::uint8:
{
attribute.uint8_map->insert({ id, value.get<UInt64>() });
break;
}
case AttributeType::uint16:
{
attribute.uint16_map->insert({ id, value.get<UInt64>() });
break;
}
case AttributeType::uint32:
{
attribute.uint32_map->insert({ id, value.get<UInt64>() });
break;
}
case AttributeType::uint64:
{
attribute.uint64_map->insert({ id, value.get<UInt64>() });
break;
}
case AttributeType::int8:
{
attribute.int8_map->insert({ id, value.get<Int64>() });
break;
}
case AttributeType::int16:
{
attribute.int16_map->insert({ id, value.get<Int64>() });
break;
}
case AttributeType::int32:
{
attribute.int32_map->insert({ id, value.get<Int64>() });
break;
}
case AttributeType::int64:
{
attribute.int64_map->insert({ id, value.get<Int64>() });
break;
}
case AttributeType::float32:
{
attribute.float32_map->insert({ id, value.get<Float64>() });
break;
}
case AttributeType::float64:
{
attribute.float64_map->insert({ id, value.get<Float64>() });
break;
}
case AttributeType::uint8: setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); break;
case AttributeType::uint16: setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); break;
case AttributeType::uint32: setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>()); break;
case AttributeType::uint64: setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>()); break;
case AttributeType::int8: setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>()); break;
case AttributeType::int16: setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>()); break;
case AttributeType::int32: setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>()); break;
case AttributeType::int64: setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>()); break;
case AttributeType::float32: setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>()); break;
case AttributeType::float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeType::string:
{
auto & map = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
const auto & string = value.get<String>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
attribute.string_map->insert({ id, StringRef{string_in_arena, string.size()} });
map.insert({ id, StringRef{string_in_arena, string.size()} });
break;
}
};
}
}
std::size_t getAttributeIndex(const std::string & attribute_name) const
const attribute_t & getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
@ -376,7 +298,7 @@ private:
ErrorCodes::BAD_ARGUMENTS
};
return it->second;
return attributes[it->second];
}
const std::string name;

View File

@ -38,6 +38,7 @@ public:
/// do not call unless you ensure that hasHierarchy() returns true
virtual id_t toParent(id_t id) const = 0;
virtual void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const = 0;
bool in(id_t child_id, const id_t ancestor_id) const
{

View File

@ -41,10 +41,10 @@ public:
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> ids) override
{
throw Exception{
"Method unsupported",
ErrorCodes::NOT_IMPLEMENTED
};
last_modification = getLastModification();
const auto query = composeLoadIdsQuery(ids);
return new MySQLBlockInputStream{pool.Get()->query(query), sample_block, max_block_size};
}
bool isModified() const override { return getLastModification() > last_modification; }
@ -76,22 +76,69 @@ private:
return mysqlxx::DateTime{std::time(nullptr)};
}
/// @todo escape table and column names
static std::string composeLoadAllQuery(const Block & block, const std::string & table)
{
std::string query{"SELECT "};
std::string query;
auto first = true;
for (const auto idx : ext::range(0, block.columns()))
{
if (!first)
query += ", ";
WriteBufferFromString out{query};
writeString("SELECT ", out);
query += block.getByPosition(idx).name;
first = false;
auto first = true;
for (const auto idx : ext::range(0, block.columns()))
{
if (!first)
writeString(", ", out);
writeString(block.getByPosition(idx).name, out);
first = false;
}
writeString(" FROM ", out);
writeProbablyBackQuotedString(table, out);
writeChar(';', out);
}
query += " FROM " + table + ';';
return query;
}
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> ids)
{
std::string query;
{
WriteBufferFromString out{query};
writeString("SELECT ", out);
auto first = true;
for (const auto idx : ext::range(0, sample_block.columns()))
{
if (!first)
writeString(", ", out);
writeString(sample_block.getByPosition(idx).name, out);
first = false;
}
const auto & id_column_name = sample_block.getByPosition(0).name;
writeString(" FROM ", out);
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
writeProbablyBackQuotedString(id_column_name, out);
writeString(" IN (", out);
first = true;
for (const auto id : ids)
{
if (!first)
writeString(", ", out);
first = false;
writeString(toString(id), out);
}
writeString(");", out);
}
return query;
}

View File

@ -22,6 +22,20 @@ public:
Context & target = ast.global ? context.getGlobalContext() : context.getSessionContext();
/** Значение readonly понимается следующим образом:
* 0 - можно всё.
* 1 - можно делать только запросы на чтение; в том числе, нельзя менять настройки.
* 2 - можно делать только запросы на чтение и можно менять настройки, кроме настройки readonly.
*/
if (context.getSettingsRef().limits.readonly == 1)
throw Exception("Cannot execute SET query in readonly mode", ErrorCodes::READONLY);
if (context.getSettingsRef().limits.readonly > 1)
for (ASTSetQuery::Changes::const_iterator it = ast.changes.begin(); it != ast.changes.end(); ++it)
if (it->name == "readonly")
throw Exception("Cannot modify 'readonly' setting in readonly mode", ErrorCodes::READONLY);
for (ASTSetQuery::Changes::const_iterator it = ast.changes.begin(); it != ast.changes.end(); ++it)
target.setSetting(it->name, it->value);
}

View File

@ -5,7 +5,6 @@
#include <DB/Parsers/ASTJoin.h>
#include <DB/Interpreters/AggregationCommon.h>
#include <DB/Interpreters/Set.h>
#include <DB/Common/Arena.h>
#include <DB/Common/HashTable/HashMap.h>
@ -67,7 +66,7 @@ public:
{
}
bool empty() { return type == Set::EMPTY; }
bool empty() { return type == Type::EMPTY; }
/** Добавить в отображение для соединения блок "правой" таблицы.
* Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
@ -155,7 +154,17 @@ private:
/// Дополнительные данные - строки, а также продолжения односвязных списков строк.
Arena pool;
Set::Type type = Set::EMPTY;
enum class Type
{
EMPTY,
KEY_64,
KEY_STRING,
HASHED,
};
Type type = Type::EMPTY;
static Type chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes);
bool keys_fit_128_bits;
Sizes key_sizes;
@ -174,7 +183,7 @@ private:
*/
mutable Poco::RWLock rwlock;
void init(Set::Type type_);
void init(Type type_);
template <ASTJoin::Strictness STRICTNESS, typename Maps>
void insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block);

View File

@ -63,7 +63,8 @@ struct Limits
M(SettingUInt64, max_ast_depth, 1000) /** Проверяются не во время парсинга, */ \
M(SettingUInt64, max_ast_elements, 10000) /** а уже после парсинга запроса. */ \
\
M(SettingBool, readonly, false) \
/** 0 - можно всё. 1 - только запросы на чтение. 2 - только запросы на чтение, а также изменение настроек, кроме настройки readonly. */ \
M(SettingUInt64, readonly, 0) \
\
/** Ограничения для максимального размера множества, получающегося при выполнении секции IN. */ \
M(SettingUInt64, max_rows_in_set, 0) \
@ -130,6 +131,22 @@ struct Limits
#undef TRY_SET
}
/// Пропустить сериализованное в бинарном виде значение из буфера.
bool tryIgnore(const String & name, ReadBuffer & buf)
{
#define TRY_IGNORE(TYPE, NAME, DEFAULT) \
else if (name == #NAME) decltype(NAME)(DEFAULT).set(buf);
if (false) {}
APPLY_FOR_LIMITS(TRY_IGNORE)
else
return false;
return true;
#undef TRY_IGNORE
}
/** Установить настройку по имени. Прочитать значение в текстовом виде из строки (например, из конфига, или из параметра URL).
*/
bool trySet(const String & name, const String & value)

View File

@ -24,6 +24,245 @@ namespace DB
{
/** Методы для разных вариантов реализации множеств.
* Используются в качестве параметра шаблона.
*/
/// Для случая, когда есть один числовой ключ.
template <typename FieldType, typename TData> /// UInt8/16/32/64 для любых типов соответствующей битности.
struct SetMethodOneNumber
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
/// Для использования одного Method в разных потоках, используйте разные State.
struct State
{
const FieldType * vec;
/** Вызывается в начале обработки каждого блока.
* Устанавливает переменные, необходимые для остальных методов, вызываемых во внутренних циклах.
*/
void init(const ConstColumnPlainPtrs & key_columns)
{
vec = &static_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData()[0];
}
/// Достать из ключевых столбцов ключ для вставки в хэш-таблицу.
Key getKey(
const ConstColumnPlainPtrs & key_columns, /// Ключевые столбцы.
size_t keys_size, /// Количество ключевых столбцов.
size_t i, /// Из какой строки блока достать ключ.
const Sizes & key_sizes) const /// Если ключи фиксированной длины - их длины. Не используется в методах по ключам переменной длины.
{
return unionCastToUInt64(vec[i]);
}
};
/** Разместить дополнительные данные, если это необходимо, в случае, когда в хэш-таблицу был вставлен новый ключ.
*/
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
};
/// Для случая, когда есть один строковый ключ.
template <typename TData>
struct SetMethodString
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
struct State
{
const ColumnString::Offsets_t * offsets;
const ColumnString::Chars_t * chars;
void init(const ConstColumnPlainPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnString & column_string = static_cast<const ColumnString &>(column);
offsets = &column_string.getOffsets();
chars = &column_string.getChars();
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]],
(i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
};
/// Для случая, когда есть один строковый ключ фиксированной длины.
template <typename TData>
struct SetMethodFixedString
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
struct State
{
size_t n;
const ColumnFixedString::Chars_t * chars;
void init(const ConstColumnPlainPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnFixedString & column_string = static_cast<const ColumnFixedString &>(column);
n = column_string.getN();
chars = &column_string.getChars();
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return StringRef(&(*chars)[i * n], n);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
};
/// Для случая, когда все ключи фиксированной длины, и они помещаются в N (например, 128) бит.
template <typename TData>
struct SetMethodKeysFixed
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
struct State
{
void init(const ConstColumnPlainPtrs & key_columns)
{
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
};
/// Для остальных случаев. По 128 битному хэшу от ключа. (При этом, строки, содержащие нули посередине, могут склеиться.)
template <typename TData>
struct SetMethodHashed
{
typedef TData Data;
typedef typename Data::key_type Key;
Data data;
struct State
{
void init(const ConstColumnPlainPtrs & key_columns)
{
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return hash128(i, keys_size, key_columns);
}
};
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
};
/** Разные варианты реализации множества.
*/
struct SetVariants
{
/// TODO Использовать для этих двух вариантов bit- или byte- set.
std::unique_ptr<SetMethodOneNumber<UInt8, HashSet<UInt8, TrivialHash, HashTableFixedGrower<8>>>> key8;
std::unique_ptr<SetMethodOneNumber<UInt16, HashSet<UInt16, TrivialHash, HashTableFixedGrower<16>>>> key16;
/** Также для эксперимента проверялась возможность использовать SmallSet,
* пока количество элементов в множестве небольшое (и, при необходимости, конвертировать в полноценный HashSet).
* Но этот эксперимент показал, что преимущество есть только в редких случаях.
*/
std::unique_ptr<SetMethodOneNumber<UInt32, HashSet<UInt32, HashCRC32<UInt32>>>> key32;
std::unique_ptr<SetMethodOneNumber<UInt64, HashSet<UInt64, HashCRC32<UInt64>>>> key64;
std::unique_ptr<SetMethodString<HashSetWithSavedHash<StringRef>>> key_string;
std::unique_ptr<SetMethodFixedString<HashSetWithSavedHash<StringRef>>> key_fixed_string;
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt128, UInt128HashCRC32>>> keys128;
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt256, UInt256HashCRC32>>> keys256;
std::unique_ptr<SetMethodHashed<HashSet<UInt128, UInt128TrivialHash>>> hashed;
/** В отличие от Aggregator, здесь не используется метод concat.
* Это сделано потому что метод hashed, хоть и медленнее, но в данном случае, использует меньше оперативки.
* так как при его использовании, сами значения ключей не сохраняются.
*/
Arena string_pool;
#define APPLY_FOR_SET_VARIANTS(M) \
M(key8) \
M(key16) \
M(key32) \
M(key64) \
M(key_string) \
M(key_fixed_string) \
M(keys128) \
M(keys256) \
M(hashed)
enum class Type
{
EMPTY,
#define M(NAME) NAME,
APPLY_FOR_SET_VARIANTS(M)
#undef M
};
Type type = Type::EMPTY;
bool empty() const { return type == Type::EMPTY; }
static Type chooseMethod(const ConstColumnPlainPtrs & key_columns, Sizes & key_sizes);
void init(Type type_);
size_t getTotalRowCount() const;
/// Считает размер в байтах буфера Set и размер string_pool'а
size_t getTotalByteCount() const;
};
/** Структура данных для реализации выражения IN.
*/
class Set
@ -37,7 +276,7 @@ public:
{
}
bool empty() { return type == EMPTY; }
bool empty() const { return data.empty(); }
/** Создать множество по выражению (для перечисления в самом запросе).
* types - типы того, что стоит слева от IN.
@ -49,11 +288,6 @@ public:
// Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
bool insertFromBlock(const Block & block, bool create_ordered_set = false);
/// Считает суммарное число ключей во всех Set'ах
size_t getTotalRowCount() const;
/// Считает суммарный размер в байтах буфферов всех Set'ов + размер string_pool'а
size_t getTotalByteCount() const;
/** Для указанных столбцов блока проверить принадлежность их значений множеству.
* Записать результат в столбец в позиции result.
*/
@ -80,43 +314,14 @@ public:
/// проверяет есть ли в Set элементы для заданного диапазона индекса
BoolMask mayBeTrueInRange(const Range & range);
enum Type
{
EMPTY = 0,
KEY_64 = 1,
KEY_STRING = 2,
HASHED = 3,
};
static Type chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes);
size_t getTotalRowCount() const { return data.getTotalRowCount(); }
size_t getTotalByteCount() const { return data.getTotalByteCount(); }
private:
/** Разные структуры данных, которые могут использоваться для проверки принадлежности
* одного или нескольких столбцов значений множеству.
*/
typedef HashSet<UInt64, HashCRC32<UInt64>> SetUInt64;
typedef HashSetWithSavedHash<StringRef> SetString;
typedef HashSet<UInt128, UInt128HashCRC32> SetHashed;
/// Специализация для случая, когда есть один числовой ключ.
std::unique_ptr<SetUInt64> key64;
/// Специализация для случая, когда есть один строковый ключ.
std::unique_ptr<SetString> key_string;
Arena string_pool;
/** Сравнивает 128 битные хэши.
* Если все ключи фиксированной длины, влезающие целиком в 128 бит, то укладывает их без изменений в 128 бит.
* Иначе - вычисляет SipHash от набора из всех ключей.
* (При этом, строки, содержащие нули посередине, могут склеиться.)
*/
std::unique_ptr<SetHashed> hashed;
Type type = EMPTY;
bool keys_fit_128_bits;
Sizes key_sizes;
SetVariants data;
/** Типы данных, из которых было создано множество.
* При проверке на принадлежность множеству, типы проверяемых столбцов должны с ними совпадать.
*/
@ -129,24 +334,7 @@ private:
size_t max_bytes;
OverflowMode overflow_mode;
void init(Type type_)
{
type = type_;
switch (type)
{
case EMPTY: break;
case KEY_64: key64 .reset(new SetUInt64); break;
case KEY_STRING: key_string .reset(new SetString); break;
case HASHED: hashed .reset(new SetHashed); break;
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
/// Если в левой части IN стоит массив. Проверяем, что хоть один элемент массива лежит в множестве.
void executeConstArray(const ColumnConstArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const;
void executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const;
/// Если в левой части набор столбцов тех же типов, что элементы множества.
@ -155,8 +343,8 @@ private:
/// Проверить не превышены ли допустимые размеры множества ключей
bool checkSetSizeLimits() const;
/// вектор упорядоченных элементов Set
/// нужен для работы индекса по первичному ключу в секции In
/// Вектор упорядоченных элементов Set.
/// Нужен для работы индекса по первичному ключу в операторе IN.
typedef std::vector<Field> OrderedSetElements;
typedef std::unique_ptr<OrderedSetElements> OrderedSetElementsPtr;
OrderedSetElementsPtr ordered_set_elements;
@ -167,6 +355,31 @@ private:
* Поэтому остальные функции по работе с множеством, не защинены.
*/
mutable Poco::RWLock rwlock;
template <typename Method>
void insertFromBlockImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
size_t rows,
SetVariants & variants);
template <typename Method>
void executeImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const;
template <typename Method>
void executeArrayImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
const ColumnArray::Offsets_t & offsets,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const;
};
typedef Poco::SharedPtr<Set> SetPtr;

View File

@ -108,7 +108,7 @@ struct Settings
M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \
\
/** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 4) \
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;
@ -126,6 +126,9 @@ struct Settings
/// Установить настройку по имени. Прочитать сериализованное в бинарном виде значение из буфера (для межсерверного взаимодействия).
void set(const String & name, ReadBuffer & buf);
/// Пропустить сериализованное в бинарном виде значение из буфера.
void ignore(const String & name, ReadBuffer & buf);
/** Установить настройку по имени. Прочитать значение в текстовом виде из строки (например, из конфига, или из параметра URL).
*/
void set(const String & name, const String & value);
@ -136,8 +139,8 @@ struct Settings
void setProfile(const String & profile_name, Poco::Util::AbstractConfiguration & config);
/// Прочитать настройки из буфера. Они записаны как набор name-value пар, идущих подряд, заканчивающихся пустым name.
/// Если выставлен флаг check_readonly, в настройках выставлено readonly, но пришли какие-то изменения кинуть исключение.
void deserialize(ReadBuffer & buf, bool check_readonly = false);
/// Если в настройках выставлено readonly=1, то игнорировать настройки.
void deserialize(ReadBuffer & buf);
/// Записать изменённые настройки в буфер. (Например, для отправки на удалённый сервер.)
void serialize(WriteBuffer & buf) const;

View File

@ -19,7 +19,7 @@ namespace DB
/// otherwise evaluate the expression
Block block{};
/** pass a dummy column name because ExpressioAnalyzer
/** pass a dummy column name because ExpressionAnalyzer
* does not work with no columns so far. */
ExpressionAnalyzer{
expr, context,

View File

@ -2,32 +2,8 @@
#include <ostream>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Parsers/IAST.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTDropQuery.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTColumnDeclaration.h>
#include <DB/Parsers/ASTAsterisk.h>
#include <DB/Parsers/ASTOrderByElement.h>
#include <DB/Parsers/ASTSubquery.h>
#include <DB/Parsers/ASTAlterQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Parsers/ASTJoin.h>
#include <DB/Parsers/ASTCheckQuery.h>
//#include <DB/Parsers/ASTMultiQuery.h>
namespace DB
@ -38,39 +14,6 @@ namespace DB
*/
void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTColumnDeclaration & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTSet & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTJoin & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTCheckQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
//void formatAST(const ASTMultiQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s,
size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
void formatAST(const ASTShowProcesslistQuery & ast, std::ostream & s,
size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
String formatColumnsForCreateQuery(NamesAndTypesList & columns);
String backQuoteIfNeed(const String & x);

View File

@ -23,7 +23,7 @@ class TableFunctionRemote : public ITableFunction
{
public:
/// Максимальное количество различных шардов и максимальное количество реплик одного шарда
const size_t MAX_ADDRESSES = 200;
const size_t MAX_ADDRESSES = 200; /// TODO Перенести в Settings.
std::string getName() const override { return "remote"; }
@ -31,8 +31,8 @@ public:
{
ASTs & args_func = typeid_cast<ASTFunction &>(*ast_function).children;
const char * err = "Table function remote requires 2 to 5 parameters: "
"addresses pattern, name of remote database, name of remote table, [username, password].";
const char * err = "Table function 'remote' requires from 2 to 5 parameters: "
"addresses pattern, name of remote database, name of remote table, [username, [password]].";
if (args_func.size() != 1)
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -42,31 +42,73 @@ public:
if (args.size() < 2 || args.size() > 5)
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
String descripton = safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[0]).value);
String remote_database = reinterpretAsIdentifier(args[1], context).name;
String remote_table = args.size() % 2 ? typeid_cast<ASTIdentifier &>(*args[2]).name : "";
String username = args.size() >= 4
? safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[args.size() - 2]).value) : "default";
String password = args.size() >= 4
? safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[args.size() - 1]).value) : "";
String description;
String remote_database;
String remote_table;
String username;
String password;
if (remote_table.empty())
size_t arg_num = 0;
auto getStringLiteral = [](const IAST & node, const char * description)
{
size_t dot = remote_database.find('.');
if (dot == String::npos)
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(&node);
if (!lit)
throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS);
if (lit->value.getType() != Field::Types::String)
throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS);
return safeGet<const String &>(lit->value);
};
description = getStringLiteral(*args[arg_num], "Hosts pattern");
++arg_num;
remote_database = reinterpretAsIdentifier(args[arg_num], context).name;
++arg_num;
size_t dot = remote_database.find('.');
if (dot != String::npos)
{
/// NOTE Плохо - не поддерживаются идентификаторы в обратных кавычках.
remote_table = remote_database.substr(dot + 1);
remote_database = remote_database.substr(0, dot);
}
else
{
if (arg_num >= args.size())
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
remote_table = reinterpretAsIdentifier(args[arg_num], context).name;
++arg_num;
}
if (arg_num < args.size())
{
username = getStringLiteral(*args[arg_num], "Username");
++arg_num;
}
else
username = "default";
if (arg_num < args.size())
{
password = getStringLiteral(*args[arg_num], "Password");
++arg_num;
}
if (arg_num < args.size())
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// В InterpreterSelectQuery будет создан ExpressionAnalzyer, который при обработке запроса наткнется на эти Identifier.
/// Нам необходимо их пометить как имя базы данных и таблицы поскольку по умолчанию стоит значение column
typeid_cast<ASTIdentifier &>(*args[1]).kind = ASTIdentifier::Database;
if (args.size() % 2)
typeid_cast<ASTIdentifier &>(*args[2]).kind = ASTIdentifier::Table;
/// Нам необходимо их пометить как имя базы данных или таблицы, поскольку по умолчанию стоит значение column.
for (auto & arg : args)
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
id->kind = ASTIdentifier::Table;
std::vector <std::vector< String> > names;
std::vector<String> shards = parseDescription(descripton, 0, descripton.size(), ',');
std::vector<std::vector<String>> names;
std::vector<String> shards = parseDescription(description, 0, description.size(), ',');
for (size_t i = 0; i < shards.size(); ++i)
names.push_back(parseDescription(shards[i], 0, shards[i].size(), '|'));

View File

@ -34,7 +34,7 @@ AggregateFunctionFactory::AggregateFunctionFactory()
/** Создать агрегатную функцию с числовым типом в параметре шаблона, в зависимости от типа аргумента.
*/
template<template <typename> class AggregateFunctionTemplate>
template <template <typename> class AggregateFunctionTemplate>
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8>;
@ -51,7 +51,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
return nullptr;
}
template<template <typename, typename> class AggregateFunctionTemplate, class Data>
template <template <typename, typename> class AggregateFunctionTemplate, class Data>
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data>;
@ -69,7 +69,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
}
template<template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data>
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data>
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data<UInt8> >;
@ -87,8 +87,45 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
}
/** Для шаблона с двумя аргументами.
*/
template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate>
static IAggregateFunction * createWithTwoNumericTypesSecond(const IDataType & second_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&second_type)) return new AggregateFunctionTemplate<FirstType, Float64>;
else
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate>
static IAggregateFunction * createWithTwoNumericTypes(const IDataType & first_type, const IDataType & second_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&first_type)) return createWithTwoNumericTypesSecond<UInt8, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeUInt16 *>(&first_type)) return createWithTwoNumericTypesSecond<UInt16, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeUInt32 *>(&first_type)) return createWithTwoNumericTypesSecond<UInt32, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeUInt64 *>(&first_type)) return createWithTwoNumericTypesSecond<UInt64, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeInt8 *>(&first_type)) return createWithTwoNumericTypesSecond<Int8, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeInt16 *>(&first_type)) return createWithTwoNumericTypesSecond<Int16, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeInt32 *>(&first_type)) return createWithTwoNumericTypesSecond<Int32, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeInt64 *>(&first_type)) return createWithTwoNumericTypesSecond<Int64, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeFloat32 *>(&first_type)) return createWithTwoNumericTypesSecond<Float32, AggregateFunctionTemplate>(second_type);
else if (typeid_cast<const DataTypeFloat64 *>(&first_type)) return createWithTwoNumericTypesSecond<Float64, AggregateFunctionTemplate>(second_type);
else
return nullptr;
}
/// min, max, any, anyLast
template<template <typename> class AggregateFunctionTemplate, template <typename> class Data>
template <template <typename> class AggregateFunctionTemplate, template <typename> class Data>
static IAggregateFunction * createAggregateFunctionSingleValue(const String & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
@ -117,6 +154,80 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na
}
/// argMin, argMax
template <template <typename> class MinMaxData, typename ResData>
static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const String & name, const IDataType & val_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt8>>>>;
else if (typeid_cast<const DataTypeUInt16 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt16>>>>;
else if (typeid_cast<const DataTypeUInt32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt32>>>>;
else if (typeid_cast<const DataTypeUInt64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<UInt64>>>>;
else if (typeid_cast<const DataTypeInt8 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int8>>>>;
else if (typeid_cast<const DataTypeInt16 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int16>>>>;
else if (typeid_cast<const DataTypeInt32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int32>>>>;
else if (typeid_cast<const DataTypeInt64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Int64>>>>;
else if (typeid_cast<const DataTypeFloat32 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Float32>>>>;
else if (typeid_cast<const DataTypeFloat64 *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Float64>>>>;
else if (typeid_cast<const DataTypeDate *>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDate::FieldType>>>>;
else if (typeid_cast<const DataTypeDateTime*>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDateTime::FieldType>>>>;
else if (typeid_cast<const DataTypeString*>(&val_type))
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataString>>>;
else
return new AggregateFunctionsArgMinMax<AggregateFunctionsArgMinMaxData<ResData, MinMaxData<SingleValueDataGeneric>>>;
}
template <template <typename> class MinMaxData>
static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & res_type = *argument_types[0];
const IDataType & val_type = *argument_types[1];
if (typeid_cast<const DataTypeUInt8 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt8>>(name, val_type);
else if (typeid_cast<const DataTypeUInt16 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt16>>(name, val_type);
else if (typeid_cast<const DataTypeUInt32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt32>>(name, val_type);
else if (typeid_cast<const DataTypeUInt64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<UInt64>>(name, val_type);
else if (typeid_cast<const DataTypeInt8 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int8>>(name, val_type);
else if (typeid_cast<const DataTypeInt16 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int16>>(name, val_type);
else if (typeid_cast<const DataTypeInt32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int32>>(name, val_type);
else if (typeid_cast<const DataTypeInt64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Int64>>(name, val_type);
else if (typeid_cast<const DataTypeFloat32 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Float32>>(name, val_type);
else if (typeid_cast<const DataTypeFloat64 *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Float64>>(name, val_type);
else if (typeid_cast<const DataTypeDate *>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDate::FieldType>>(name, val_type);
else if (typeid_cast<const DataTypeDateTime*>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDateTime::FieldType>>(name, val_type);
else if (typeid_cast<const DataTypeString*>(&res_type))
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(name, val_type);
else
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric>(name, val_type);
}
AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const
{
if (name == "count")
@ -130,9 +241,9 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
else if (name == "max")
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types);
else if (name == "argMin")
return new AggregateFunctionArgMin;
return createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types);
else if (name == "argMax")
return new AggregateFunctionArgMax;
return createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types);
else if (name == "groupArray")
return new AggregateFunctionGroupArray;
else if (name == "groupUniqArray")
@ -326,6 +437,32 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
return res;
}
else if (name == "medianTimingWeighted" || name == "quantileTimingWeighted")
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantileTimingWeighted>(*argument_types[0], *argument_types[1]);
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
else if (name == "quantilesTimingWeighted")
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantilesTimingWeighted>(*argument_types[0], *argument_types[1]);
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
else if (name == "quantileDeterministic")
{
if (argument_types.size() != 2)
@ -486,6 +623,9 @@ bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int
"medianTiming",
"quantileTiming",
"quantilesTiming",
"quantileTimingWeighted",
"quantilesTimingWeighted",
"medianTimingWeighted",
"quantileDeterministic",
"quantilesDeterministic",
nullptr

View File

@ -40,6 +40,12 @@
#include <DB/DataStreams/AsynchronousBlockInputStream.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTQueryWithOutput.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Interpreters/Context.h>
@ -123,9 +129,12 @@ private:
/// Распарсенный запрос. Оттуда берутся некоторые настройки (формат).
ASTPtr parsed_query;
/// Последнее полученное от сервера исключение.
/// Последнее полученное от сервера исключение. Для кода возврата в неинтерактивном режиме.
ExceptionPtr last_exception;
/// Было ли в последнем запросе исключение.
bool got_exception = false;
Stopwatch watch;
/// С сервера периодически приходит информация, о том, сколько прочитано данных за прошедшее время.
@ -356,7 +365,7 @@ private:
bool ends_with_semicolon = line[ws - 1] == ';';
bool ends_with_backslash = line[ws - 1] == '\\';
has_vertical_output_suffix = (ws >= 2) && (line[ws - 2] == '\\') && (line[ws - 1] == 'G');
if (ends_with_backslash)
@ -370,7 +379,7 @@ private:
{
// Заменяем переводы строк на пробелы, а то возникает следуцющая проблема.
// Каждая строчка многострочного запроса сохраняется в истории отдельно. Если
// выйти из клиента и войти заново, то при нажатии клавиши "вверх" выводится не
// выйти из клиента и войти заново, то при нажатии клавиши "вверх" выводится не
// весь многострочный запрос, а каждая его строчка по-отдельности.
std::string logged_query = query;
std::replace(logged_query.begin(), logged_query.end(), '\n', ' ');
@ -479,6 +488,7 @@ private:
return false;
resetOutput();
got_exception = false;
watch.restart();
@ -512,25 +522,29 @@ private:
else
processOrdinaryQuery();
if (set_query)
/// В случае исключения, не будем менять контекст (текущая БД, настройки) на клиенте.
if (!got_exception)
{
/// Запоминаем все изменения в настройках, чтобы не потерять их при разрыве соединения.
for (ASTSetQuery::Changes::const_iterator it = set_query->changes.begin(); it != set_query->changes.end(); ++it)
if (set_query)
{
if (it->name == "profile")
current_profile = it->value.safeGet<String>();
else
context.setSetting(it->name, it->value);
/// Запоминаем все изменения в настройках, чтобы не потерять их при разрыве соединения.
for (ASTSetQuery::Changes::const_iterator it = set_query->changes.begin(); it != set_query->changes.end(); ++it)
{
if (it->name == "profile")
current_profile = it->value.safeGet<String>();
else
context.setSetting(it->name, it->value);
}
}
}
if (use_query)
{
const String & new_database = use_query->database;
/// Если клиент инициирует пересоединение, он берет настройки из конфига
config().setString("database", new_database);
/// Если connection инициирует пересоединение, он использует свою переменную
connection->setDefaultDatabase(new_database);
if (use_query)
{
const String & new_database = use_query->database;
/// Если клиент инициирует пересоединение, он берет настройки из конфига
config().setString("database", new_database);
/// Если connection инициирует пересоединение, он использует свою переменную
connection->setDefaultDatabase(new_database);
}
}
if (is_interactive)
@ -791,7 +805,7 @@ private:
onException(*packet.exception);
last_exception = packet.exception;
return false;
default:
throw Exception("Unexpected packet from server (expected Data, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
@ -811,7 +825,7 @@ private:
if (!block_std_out)
{
String current_format = format;
/// Формат может быть указан в запросе.
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
{
@ -823,10 +837,10 @@ private:
current_format = id->name;
}
}
if (has_vertical_output_suffix)
current_format = "Vertical";
block_std_out = context.getFormatFactory().getOutput(current_format, std_out, block);
block_std_out->writePrefix();
}
@ -959,6 +973,7 @@ private:
void onException(const Exception & e)
{
resetOutput();
got_exception = true;
std::cerr << "Received exception from server:" << std::endl
<< "Code: " << e.code() << ". " << e.displayText();

View File

@ -1,3 +1,5 @@
#include <DB/Interpreters/Set.h>
#include <DB/Interpreters/Join.h>
#include <DB/DataStreams/CreatingSetsBlockInputStream.h>
#include <iomanip>

View File

@ -25,6 +25,8 @@
#include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Interpreters/Context.h>
using Poco::SharedPtr;

View File

@ -5,6 +5,7 @@
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/IO/copyData.h>

View File

@ -8,6 +8,7 @@
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/ASTAlterQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTCheckQuery.h>
@ -82,7 +83,7 @@ void InterpreterQuery::execute(WriteBuffer & ostr, ReadBuffer * remaining_data_i
}
else if (typeid_cast<ASTSetQuery *>(&*query_ptr))
{
throwIfReadOnly();
/// readonly проверяется внутри InterpreterSetQuery
InterpreterSetQuery interpreter(query_ptr, context);
interpreter.execute();
}
@ -174,7 +175,7 @@ BlockIO InterpreterQuery::execute()
}
else if (typeid_cast<ASTSetQuery *>(&*query_ptr))
{
throwIfReadOnly();
/// readonly проверяется внутри InterpreterSetQuery
InterpreterSetQuery interpreter(query_ptr, context);
interpreter.execute();
}

View File

@ -8,6 +8,7 @@
#include <DB/IO/copyData.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/formatAST.h>

View File

@ -9,15 +9,53 @@ namespace DB
{
Join::Type Join::chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes)
{
size_t keys_size = key_columns.size();
keys_fit_128_bits = true;
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j)
{
if (!key_columns[j]->isFixed())
{
keys_fit_128_bits = false;
break;
}
key_sizes[j] = key_columns[j]->sizeOfField();
keys_bytes += key_sizes[j];
}
if (keys_bytes > 16)
keys_fit_128_bits = false;
/// Если есть один числовой ключ, который помещается в 64 бита
if (keys_size == 1 && key_columns[0]->isNumeric())
return Type::KEY_64;
/// Если есть один строковый ключ, то используем хэш-таблицу с ним
if (keys_size == 1
&& (typeid_cast<const ColumnString *>(key_columns[0])
|| typeid_cast<const ColumnConstString *>(key_columns[0])
|| (typeid_cast<const ColumnFixedString *>(key_columns[0]) && !keys_fit_128_bits)))
return Type::KEY_STRING;
/// Если много ключей - будем строить множество хэшей от них
return Type::HASHED;
}
template <typename Maps>
static void initImpl(Maps & maps, Set::Type type)
static void initImpl(Maps & maps, Join::Type type)
{
switch (type)
{
case Set::EMPTY: break;
case Set::KEY_64: maps.key64 .reset(new typename Maps::MapUInt64); break;
case Set::KEY_STRING: maps.key_string .reset(new typename Maps::MapString); break;
case Set::HASHED: maps.hashed .reset(new typename Maps::MapHashed); break;
case Join::Type::EMPTY: break;
case Join::Type::KEY_64: maps.key64 .reset(new typename Maps::MapUInt64); break;
case Join::Type::KEY_STRING: maps.key_string .reset(new typename Maps::MapString); break;
case Join::Type::HASHED: maps.hashed .reset(new typename Maps::MapHashed); break;
default:
throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
@ -51,7 +89,7 @@ static size_t getTotalByteCountImpl(const Maps & maps)
}
void Join::init(Set::Type type_)
void Join::init(Type type_)
{
type = type_;
@ -190,7 +228,7 @@ struct Inserter<ASTJoin::All, Join::MapsAll::MapString>
template <ASTJoin::Strictness STRICTNESS, typename Maps>
void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block)
{
if (type == Set::KEY_64)
if (type == Type::KEY_64)
{
typedef typename Maps::MapUInt64 Map;
Map & res = *maps.key64;
@ -204,7 +242,7 @@ void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainP
Inserter<STRICTNESS, Map>::insert(res, key, stored_block, i, pool);
}
}
else if (type == Set::KEY_STRING)
else if (type == Type::KEY_STRING)
{
typedef typename Maps::MapString Map;
Map & res = *maps.key_string;
@ -239,7 +277,7 @@ void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainP
else
throw Exception("Illegal type of column when creating join with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else if (type == Set::HASHED)
else if (type == Type::HASHED)
{
typedef typename Maps::MapHashed Map;
Map & res = *maps.hashed;
@ -274,7 +312,7 @@ bool Join::insertFromBlock(const Block & block)
/// Какую структуру данных для множества использовать?
if (empty())
init(Set::chooseMethod(key_columns, keys_fit_128_bits, key_sizes));
init(chooseMethod(key_columns, keys_fit_128_bits, key_sizes));
blocks.push_back(block);
Block * stored_block = &blocks.back();
@ -441,7 +479,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
if (strictness == ASTJoin::All)
offsets_to_replicate.reset(new IColumn::Offsets_t(rows));
if (type == Set::KEY_64)
if (type == Type::KEY_64)
{
typedef typename Maps::MapUInt64 Map;
const Map & map = *maps.key64;
@ -455,7 +493,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
Adder<KIND, STRICTNESS, Map>::add(map, key, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get());
}
}
else if (type == Set::KEY_STRING)
else if (type == Type::KEY_STRING)
{
typedef typename Maps::MapString Map;
const Map & map = *maps.key_string;
@ -490,7 +528,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
else
throw Exception("Illegal type of column when creating set with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else if (type == Set::HASHED)
else if (type == Type::HASHED)
{
typedef typename Maps::MapHashed Map;
Map & map = *maps.hashed;

View File

@ -21,78 +21,144 @@
namespace DB
{
size_t Set::getTotalRowCount() const
void SetVariants::init(Type type_)
{
size_t rows = 0;
if (key64)
rows += key64->size();
if (key_string)
rows += key_string->size();
if (hashed)
rows += hashed->size();
return rows;
type = type_;
switch (type)
{
case Type::EMPTY: break;
#define M(NAME) \
case Type::NAME: NAME.reset(new decltype(NAME)::element_type); break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
size_t Set::getTotalByteCount() const
size_t SetVariants::getTotalRowCount() const
{
size_t bytes = 0;
if (key64)
bytes += key64->getBufferSizeInBytes();
if (key_string)
bytes += key_string->getBufferSizeInBytes();
if (hashed)
bytes += hashed->getBufferSizeInBytes();
bytes += string_pool.size();
return bytes;
switch (type)
{
case Type::EMPTY: return 0;
#define M(NAME) \
case Type::NAME: return NAME->data.size();
APPLY_FOR_SET_VARIANTS(M)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
size_t SetVariants::getTotalByteCount() const
{
switch (type)
{
case Type::EMPTY: return 0;
#define M(NAME) \
case Type::NAME: return NAME->data.getBufferSizeInBytes();
APPLY_FOR_SET_VARIANTS(M)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
bool Set::checkSetSizeLimits() const
{
if (max_rows && getTotalRowCount() > max_rows)
if (max_rows && data.getTotalRowCount() > max_rows)
return false;
if (max_bytes && getTotalByteCount() > max_bytes)
if (max_bytes && data.getTotalByteCount() > max_bytes)
return false;
return true;
}
Set::Type Set::chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes)
SetVariants::Type SetVariants::chooseMethod(const ConstColumnPlainPtrs & key_columns, Sizes & key_sizes)
{
size_t keys_size = key_columns.size();
keys_fit_128_bits = true;
bool all_fixed = true;
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j)
{
if (!key_columns[j]->isFixed())
{
keys_fit_128_bits = false;
all_fixed = false;
break;
}
key_sizes[j] = key_columns[j]->sizeOfField();
keys_bytes += key_sizes[j];
}
if (keys_bytes > 16)
keys_fit_128_bits = false;
/// Если есть один числовой ключ, который помещается в 64 бита
if (keys_size == 1 && key_columns[0]->isNumeric())
return KEY_64;
{
size_t size_of_field = key_columns[0]->sizeOfField();
if (size_of_field == 1)
return SetVariants::Type::key8;
if (size_of_field == 2)
return SetVariants::Type::key16;
if (size_of_field == 4)
return SetVariants::Type::key32;
if (size_of_field == 8)
return SetVariants::Type::key64;
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8.", ErrorCodes::LOGICAL_ERROR);
}
/// Если ключи помещаются в N бит, будем использовать хэш-таблицу по упакованным в N-бит ключам
if (all_fixed && keys_bytes <= 16)
return SetVariants::Type::keys128;
if (all_fixed && keys_bytes <= 32)
return SetVariants::Type::keys256;
/// Если есть один строковый ключ, то используем хэш-таблицу с ним
if (keys_size == 1
&& (typeid_cast<const ColumnString *>(key_columns[0])
|| typeid_cast<const ColumnConstString *>(key_columns[0])
|| (typeid_cast<const ColumnFixedString *>(key_columns[0]) && !keys_fit_128_bits)))
return KEY_STRING;
if (keys_size == 1 && (typeid_cast<const ColumnString *>(key_columns[0]) || typeid_cast<const ColumnConstString *>(key_columns[0])))
return SetVariants::Type::key_string;
/// Если много ключей - будем строить множество хэшей от них
return HASHED;
if (keys_size == 1 && typeid_cast<const ColumnFixedString *>(key_columns[0]))
return SetVariants::Type::key_fixed_string;
/// Иначе будем агрегировать по конкатенации ключей.
return SetVariants::Type::hashed;
}
template <typename Method>
void NO_INLINE Set::insertFromBlockImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
size_t rows,
SetVariants & variants)
{
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes);
typename Method::Data::iterator it = method.data.find(key);
bool inserted;
method.data.emplace(key, it, inserted);
if (inserted)
method.onNewKey(*it, keys_size, i, variants.string_pool);
}
}
@ -115,94 +181,28 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
/// Какую структуру данных для множества использовать?
if (empty())
init(chooseMethod(key_columns, keys_fit_128_bits, key_sizes));
data.init(data.chooseMethod(key_columns, key_sizes));
if (type == KEY_64)
{
SetUInt64 & res = *key64;
const IColumn & column = *key_columns[0];
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
UInt64 key = column.get64(i);
res.insert(key);
if(create_ordered_set)
ordered_set_elements->push_back(column[i]);
}
}
else if (type == KEY_STRING)
{
SetString & res = *key_string;
const IColumn & column = *key_columns[0];
if (const ColumnString * column_string = typeid_cast<const ColumnString *>(&column))
{
const ColumnString::Offsets_t & offsets = column_string->getOffsets();
const ColumnString::Chars_t & data = column_string->getChars();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
StringRef ref(&data[i == 0 ? 0 : offsets[i - 1]], (i == 0 ? offsets[i] : (offsets[i] - offsets[i - 1])) - 1);
SetString::iterator it;
bool inserted;
res.emplace(ref, it, inserted);
if (inserted)
it->data = string_pool.insert(ref.data, ref.size);
if(create_ordered_set)
ordered_set_elements->push_back(std::string(ref.data, ref.size));
}
}
else if (const ColumnFixedString * column_string = typeid_cast<const ColumnFixedString *>(&column))
{
size_t n = column_string->getN();
const ColumnFixedString::Chars_t & data = column_string->getChars();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
StringRef ref(&data[i * n], n);
SetString::iterator it;
bool inserted;
res.emplace(ref, it, inserted);
if (inserted)
it->data = string_pool.insert(ref.data, ref.size);
if(create_ordered_set)
ordered_set_elements->push_back(std::string(ref.data, ref.size));
}
}
else
throw Exception("Illegal type of column when creating set with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else if (type == HASHED)
{
SetHashed & res = *hashed;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
res.insert(keys_fit_128_bits ? packFixed<UInt128>(i, keys_size, key_columns, key_sizes) : hash128(i, keys_size, key_columns));
}
if (false) {}
#define M(NAME) \
else if (data.type == SetVariants::Type::NAME) \
insertFromBlockImpl(*data.NAME, key_columns, rows, data);
APPLY_FOR_SET_VARIANTS(M)
#undef M
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
if (create_ordered_set)
for (size_t i = 0; i < rows; ++i)
ordered_set_elements->push_back((*key_columns[0])[i]); /// ordered_set для индекса работает только если IN одному ключу.
if (!checkSetSizeLimits())
{
if (overflow_mode == OverflowMode::THROW)
throw Exception("IN-Set size exceeded."
" Rows: " + toString(getTotalRowCount()) +
throw Exception("IN-set size exceeded."
" Rows: " + toString(data.getTotalRowCount()) +
", limit: " + toString(max_rows) +
". Bytes: " + toString(getTotalByteCount()) +
". Bytes: " + toString(data.getTotalByteCount()) +
", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
@ -417,12 +417,19 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result,
throw Exception(std::string() + "Types in section IN don't match: " + data_types[0]->getName() + " on the right, " + array_type->getNestedType()->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH);
IColumn * in_column = &*block.getByPosition(arguments[0]).column;
if (ColumnConstArray * col = typeid_cast<ColumnConstArray *>(in_column))
executeConstArray(col, vec_res, negative);
else if (ColumnArray * col = typeid_cast<ColumnArray *>(in_column))
/// Константный столбец слева от IN поддерживается не напрямую. Для этого, он сначала материализуется.
ColumnPtr materialized_column;
if (in_column->isConst())
{
materialized_column = static_cast<const IColumnConst *>(in_column)->convertToFullColumn();
in_column = materialized_column.get();
}
if (ColumnArray * col = typeid_cast<ColumnArray *>(in_column))
executeArray(col, vec_res, negative);
else
throw Exception("Unexpeced array column type: " + in_column->getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Unexpected array column type: " + in_column->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else
{
@ -439,82 +446,88 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result,
throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: " + data_types[i]->getName() + " on the right, " + block.getByPosition(arguments[i]).type->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH);
}
/// Константные столбцы слева от IN поддерживается не напрямую. Для этого, они сначала материализуется.
Columns materialized_columns;
for (auto & column_ptr : key_columns)
{
if (column_ptr->isConst())
{
materialized_columns.emplace_back(static_cast<const IColumnConst *>(column_ptr)->convertToFullColumn());
column_ptr = materialized_columns.back().get();
}
}
executeOrdinary(key_columns, vec_res, negative);
}
}
void Set::executeOrdinary(const ConstColumnPlainPtrs & key_columns, ColumnUInt8::Container_t & vec_res, bool negative) const
template <typename Method>
void NO_INLINE Set::executeImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const
{
size_t keys_size = data_types.size();
size_t rows = key_columns[0]->size();
Row key(keys_size);
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
if (type == KEY_64)
/// NOTE Не используется оптимизация для подряд идущих одинаковых значений.
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
const SetUInt64 & set = *key64;
const IColumn & column = *key_columns[0];
/// Строим ключ
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes);
vec_res[i] = negative ^ (method.data.end() != method.data.find(key));
}
}
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
template <typename Method>
void NO_INLINE Set::executeArrayImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
const ColumnArray::Offsets_t & offsets,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const
{
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
UInt64 key = column.get64(i);
vec_res[i] = negative ^ (set.end() != set.find(key));
typename Method::Key key = state.getKey(key_columns, keys_size, j, key_sizes);
res |= negative ^ (method.data.end() != method.data.find(key));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
else if (type == KEY_STRING)
{
const SetString & set = *key_string;
const IColumn & column = *key_columns[0];
}
if (const ColumnString * column_string = typeid_cast<const ColumnString *>(&column))
{
const ColumnString::Offsets_t & offsets = column_string->getOffsets();
const ColumnString::Chars_t & data = column_string->getChars();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
StringRef ref(&data[i == 0 ? 0 : offsets[i - 1]], (i == 0 ? offsets[i] : (offsets[i] - offsets[i - 1])) - 1);
vec_res[i] = negative ^ (set.end() != set.find(ref));
}
}
else if (const ColumnFixedString * column_string = typeid_cast<const ColumnFixedString *>(&column))
{
size_t n = column_string->getN();
const ColumnFixedString::Chars_t & data = column_string->getChars();
void Set::executeOrdinary(const ConstColumnPlainPtrs & key_columns, ColumnUInt8::Container_t & vec_res, bool negative) const
{
size_t rows = key_columns[0]->size();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
StringRef ref(&data[i * n], n);
vec_res[i] = negative ^ (set.end() != set.find(ref));
}
}
else if (const ColumnConstString * column_string = typeid_cast<const ColumnConstString *>(&column))
{
bool res = negative ^ (set.end() != set.find(StringRef(column_string->getData())));
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
vec_res[i] = res;
}
else
throw Exception("Illegal type of column when creating set with string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else if (type == HASHED)
{
const SetHashed & set = *hashed;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
vec_res[i] = negative ^
(set.end() != set.find(keys_fit_128_bits
? packFixed<UInt128>(i, keys_size, key_columns, key_sizes)
: hash128(i, keys_size, key_columns)));
}
if (false) {}
#define M(NAME) \
else if (data.type == SetVariants::Type::NAME) \
executeImpl(*data.NAME, key_columns, vec_res, negative, rows);
APPLY_FOR_SET_VARIANTS(M)
#undef M
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
}
@ -525,150 +538,16 @@ void Set::executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t
const ColumnArray::Offsets_t & offsets = key_column->getOffsets();
const IColumn & nested_column = key_column->getData();
if (type == KEY_64)
{
const SetUInt64 & set = *key64;
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
UInt64 key = nested_column.get64(j);
res |= negative ^ (set.end() != set.find(key));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
}
else if (type == KEY_STRING)
{
const SetString & set = *key_string;
if (const ColumnString * column_string = typeid_cast<const ColumnString *>(&nested_column))
{
const ColumnString::Offsets_t & nested_offsets = column_string->getOffsets();
const ColumnString::Chars_t & data = column_string->getChars();
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
size_t begin = j == 0 ? 0 : nested_offsets[j - 1];
size_t end = nested_offsets[j];
StringRef ref(&data[begin], end - begin - 1);
res |= negative ^ (set.end() != set.find(ref));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
}
else if (const ColumnFixedString * column_string = typeid_cast<const ColumnFixedString *>(&nested_column))
{
size_t n = column_string->getN();
const ColumnFixedString::Chars_t & data = column_string->getChars();
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
StringRef ref(&data[j * n], n);
res |= negative ^ (set.end() != set.find(ref));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
}
else
throw Exception("Illegal type of column when looking for Array(String) key: " + nested_column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else if (type == HASHED)
{
const SetHashed & set = *hashed;
ConstColumnPlainPtrs nested_columns(1, &nested_column);
size_t prev_offset = 0;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
res |= negative ^
(set.end() != set.find(keys_fit_128_bits
? packFixed<UInt128>(i, 1, nested_columns, key_sizes)
: hash128(i, 1, nested_columns)));
if (res)
break;
}
vec_res[i] = res;
prev_offset = offsets[i];
}
}
if (false) {}
#define M(NAME) \
else if (data.type == SetVariants::Type::NAME) \
executeArrayImpl(*data.NAME, ConstColumnPlainPtrs{&nested_column}, offsets, vec_res, negative, rows);
APPLY_FOR_SET_VARIANTS(M)
#undef M
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
}
void Set::executeConstArray(const ColumnConstArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const
{
if (type == HASHED)
{
ColumnPtr full_column = key_column->convertToFullColumn();
executeArray(typeid_cast<ColumnArray *>(&*full_column), vec_res, negative);
return;
}
size_t rows = key_column->size();
Array values = key_column->getData();
UInt8 res = 0;
/// Для всех элементов
for (size_t j = 0; j < values.size(); ++j)
{
if (type == KEY_64)
{
const SetUInt64 & set = *key64;
UInt64 key = get<UInt64>(values[j]);
res |= negative ^ (set.end() != set.find(key));
}
else if (type == KEY_STRING)
{
const SetString & set = *key_string;
res |= negative ^ (set.end() != set.find(StringRef(get<String>(values[j]))));
}
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
if (res)
break;
}
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
vec_res[i] = res;
}
BoolMask Set::mayBeTrueInRange(const Range & range)
{

View File

@ -32,6 +32,20 @@ void Settings::set(const String & name, ReadBuffer & buf)
#undef TRY_SET
}
/// Пропустить сериализованное в бинарном виде значение из буфера.
void Settings::ignore(const String & name, ReadBuffer & buf)
{
#define TRY_IGNORE(TYPE, NAME, DEFAULT) \
else if (name == #NAME) decltype(NAME)(DEFAULT).set(buf);
if (false) {}
APPLY_FOR_SETTINGS(TRY_IGNORE)
else if (!limits.tryIgnore(name, buf))
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_IGNORE
}
/** Установить настройку по имени. Прочитать значение в текстовом виде из строки (например, из конфига, или из параметра URL).
*/
void Settings::set(const String & name, const String & value)
@ -71,9 +85,9 @@ void Settings::setProfile(const String & profile_name, Poco::Util::AbstractConfi
/// Прочитать настройки из буфера. Они записаны как набор name-value пар, идущих подряд, заканчивающихся пустым name.
/// Если выставлен флаг check_readonly, в настройках выставлено readonly, но пришли какие-то изменения кинуть исключение.
void Settings::deserialize(ReadBuffer & buf, bool check_readonly)
void Settings::deserialize(ReadBuffer & buf)
{
bool readonly = limits.readonly;
bool readonly = limits.readonly == 1; /// Если readonly = 2, то можно менять настройки.
while (true)
{
@ -84,10 +98,10 @@ void Settings::deserialize(ReadBuffer & buf, bool check_readonly)
if (name.empty())
break;
if (check_readonly && readonly)
throw Exception("Can't set setting " + name + ". Settings are readonly.", ErrorCodes::READONLY);
set(name, buf);
if (!readonly)
set(name, buf);
else
ignore(name, buf);
}
}

View File

@ -3,6 +3,8 @@
#include <DB/Parsers/formatAST.h>
#include <DB/DataStreams/BlockIO.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Interpreters/executeQuery.h>

View File

@ -1,7 +1,5 @@
#include <sstream>
#include <boost/variant/static_visitor.hpp>
#include <mysqlxx/Manip.h>
#include <DB/IO/WriteBufferFromOStream.h>
@ -12,6 +10,32 @@
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTDropQuery.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTRenameQuery.h>
#include <DB/Parsers/ASTShowTablesQuery.h>
#include <DB/Parsers/ASTUseQuery.h>
#include <DB/Parsers/ASTSetQuery.h>
#include <DB/Parsers/ASTOptimizeQuery.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTColumnDeclaration.h>
#include <DB/Parsers/ASTAsterisk.h>
#include <DB/Parsers/ASTOrderByElement.h>
#include <DB/Parsers/ASTSubquery.h>
#include <DB/Parsers/ASTAlterQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Parsers/ASTJoin.h>
#include <DB/Parsers/ASTCheckQuery.h>
//#include <DB/Parsers/ASTMultiQuery.h>
#include <DB/Parsers/formatAST.h>
@ -45,52 +69,6 @@ String hightlight(const String & keyword, const String & color_sequence, const b
}
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
#define DISPATCH(NAME) \
else if (const AST ## NAME * concrete = typeid_cast<const AST ## NAME *>(&ast)) \
formatAST(*concrete, s, indent, hilite, one_line, need_parens);
if (false) {}
DISPATCH(SelectQuery)
DISPATCH(InsertQuery)
DISPATCH(CreateQuery)
DISPATCH(DropQuery)
DISPATCH(RenameQuery)
DISPATCH(ShowTablesQuery)
DISPATCH(UseQuery)
DISPATCH(SetQuery)
DISPATCH(OptimizeQuery)
DISPATCH(ExistsQuery)
DISPATCH(ShowCreateQuery)
DISPATCH(DescribeQuery)
DISPATCH(ExpressionList)
DISPATCH(Function)
DISPATCH(Identifier)
DISPATCH(Literal)
DISPATCH(NameTypePair)
DISPATCH(ColumnDeclaration)
DISPATCH(Asterisk)
DISPATCH(OrderByElement)
DISPATCH(Subquery)
DISPATCH(AlterQuery)
DISPATCH(ShowProcesslistQuery)
DISPATCH(Set)
DISPATCH(Join)
DISPATCH(CheckQuery)
// DISPATCH(MultiQuery)
else
throw Exception("Unknown element in AST: " + ast.getID()
+ ((ast.range.first && (ast.range.second > ast.range.first))
? " '" + std::string(ast.range.first, ast.range.second - ast.range.first) + "'"
: ""),
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
#undef DISPATCH
}
void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it)
@ -354,17 +332,17 @@ void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::os
void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "EXISTS TABLE", s, indent, hilite, one_line);
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "EXISTS TABLE", s, indent, hilite, one_line, false);
}
void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "DESCRIBE TABLE", s, indent, hilite, one_line);
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "DESCRIBE TABLE", s, indent, hilite, one_line, false);
}
void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "SHOW CREATE TABLE", s, indent, hilite, one_line);
formatAST(static_cast<const ASTQueryWithTableAndOutput &>(ast), "SHOW CREATE TABLE", s, indent, hilite, one_line, false);
}
void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
@ -888,6 +866,52 @@ void formatAST(const ASTMultiQuery & ast, std::ostream & s, size_t indent, bool
}*/
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens)
{
#define DISPATCH(NAME) \
else if (const AST ## NAME * concrete = typeid_cast<const AST ## NAME *>(&ast)) \
formatAST(*concrete, s, indent, hilite, one_line, need_parens);
if (false) {}
DISPATCH(SelectQuery)
DISPATCH(InsertQuery)
DISPATCH(CreateQuery)
DISPATCH(DropQuery)
DISPATCH(RenameQuery)
DISPATCH(ShowTablesQuery)
DISPATCH(UseQuery)
DISPATCH(SetQuery)
DISPATCH(OptimizeQuery)
DISPATCH(ExistsQuery)
DISPATCH(ShowCreateQuery)
DISPATCH(DescribeQuery)
DISPATCH(ExpressionList)
DISPATCH(Function)
DISPATCH(Identifier)
DISPATCH(Literal)
DISPATCH(NameTypePair)
DISPATCH(ColumnDeclaration)
DISPATCH(Asterisk)
DISPATCH(OrderByElement)
DISPATCH(Subquery)
DISPATCH(AlterQuery)
DISPATCH(ShowProcesslistQuery)
DISPATCH(Set)
DISPATCH(Join)
DISPATCH(CheckQuery)
// DISPATCH(MultiQuery)
else
throw Exception("Unknown element in AST: " + ast.getID()
+ ((ast.range.first && (ast.range.second > ast.range.first))
? " '" + std::string(ast.range.first, ast.range.second - ast.range.first) + "'"
: ""),
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
#undef DISPATCH
}
String formatColumnsForCreateQuery(NamesAndTypesList & columns)
{
std::string res;

View File

@ -539,7 +539,7 @@ void TCPHandler::receiveQuery()
/// Настройки на отдельный запрос.
if (client_revision >= DBMS_MIN_REVISION_WITH_PER_QUERY_SETTINGS)
{
query_context.getSettingsRef().deserialize(*in, true);
query_context.getSettingsRef().deserialize(*in);
}
readVarUInt(stage, *in);

View File

@ -101,8 +101,14 @@ BlockInputStreams StorageBuffer::read(
BlockInputStreams streams_from_dst;
if (!no_destination)
streams_from_dst = context.getTable(destination_database, destination_table)->read(
column_names, query, context, settings, processed_stage, max_block_size, threads);
{
auto destination = context.getTable(destination_database, destination_table);
if (destination.get() == this)
throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP);
streams_from_dst = destination->read(column_names, query, context, settings, processed_stage, max_block_size, threads);
}
BlockInputStreams streams_from_buffers;
streams_from_buffers.reserve(num_shards);
@ -158,6 +164,9 @@ public:
{
destination = storage.context.tryGetTable(storage.destination_database, storage.destination_table);
if (destination.get() == &storage)
throw Exception("Destination table is myself. Write will cause infinite loop.", ErrorCodes::INFINITE_LOOP);
/// Проверяем структуру таблицы.
try
{
@ -232,9 +241,9 @@ private:
if (!storage.no_destination)
{
auto destination = storage.context.tryGetTable(storage.destination_database, storage.destination_table);
appendBlock(sorted_block, block_to_write);
storage.writeBlockToDestination(block_to_write,
storage.context.tryGetTable(storage.destination_database, storage.destination_table));
storage.writeBlockToDestination(block_to_write, destination);
}
}
else

View File

@ -6,6 +6,7 @@
#include <DB/Storages/Distributed/DistributedBlockOutputStream.h>
#include <DB/Storages/Distributed/DirectoryMonitor.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/InterpreterAlterQuery.h>

View File

@ -8,6 +8,7 @@
#include <DB/IO/ReadBufferFromString.h>
#include <DB/Interpreters/InterpreterAlterQuery.h>
#include <DB/Common/VirtualColumnUtils.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
#include <time.h>

View File

@ -12,7 +12,7 @@ TableFunctionPtr TableFunctionFactory::get(
const String & name,
const Context & context) const
{
if (context.getSettings().limits.readonly)
if (context.getSettings().limits.readonly == 1) /** Например, для readonly = 2 - разрешено. */
throw Exception("Table functions are forbidden in readonly mode", ErrorCodes::READONLY);
if (name == "merge") return new TableFunctionMerge;

View File

@ -0,0 +1,4 @@
50 54
50 77
50 nan
[50,90] [54,89]

View File

@ -0,0 +1,4 @@
SELECT medianTiming(t), medianTimingWeighted(t, w) FROM (SELECT number AS t, number = 77 ? 10 : 1 AS w FROM system.numbers LIMIT 100);
SELECT quantileTiming(0.5)(t), quantileTimingWeighted(0.5)(t, w) FROM (SELECT number AS t, number = 77 ? 10 : 0 AS w FROM system.numbers LIMIT 100);
SELECT medianTiming(t), medianTimingWeighted(t, w) FROM (SELECT number AS t, number = 77 ? 0 : 0 AS w FROM system.numbers LIMIT 100);
SELECT quantilesTiming(0.5, 0.9)(t), quantilesTimingWeighted(0.5, 0.9)(t, w) FROM (SELECT number AS t, number = 77 ? 10 : 1 AS w FROM system.numbers LIMIT 100);

View File

@ -0,0 +1,18 @@
DROP TABLE IF EXISTS test.mt;
DROP TABLE IF EXISTS test.buf;
CREATE TABLE test.mt (a UInt8, d Date) ENGINE = MergeTree(d, a, 1);
CREATE TABLE test.buf AS test.mt ENGINE = Buffer(test, mt, 1, 100, 100, 1000000, 1000000, 1000000000, 1000000000);
INSERT INTO test.buf SELECT toUInt8(number) AS a, toDate('2015-01-01') AS d FROM system.numbers LIMIT 1000;
SELECT count() FROM test.mt;
SELECT count() FROM test.buf;
SELECT * FROM test.buf PREWHERE a < 10;
OPTIMIZE TABLE test.buf;
SELECT count() FROM test.mt;
SELECT count() FROM test.buf;
SELECT * FROM test.buf PREWHERE a < 10;
DROP TABLE test.buf;
DROP TABLE test.mt;

View File

@ -0,0 +1 @@
1

View File

@ -0,0 +1 @@
SELECT (1, '') IN ((1, ''));

View File

@ -0,0 +1,27 @@
1
1
1
1
1
1
1
0
0
1
0
0
1
0
0
1
0
1
0
1
1
0
0
1
1
0
1

View File

@ -0,0 +1,21 @@
SELECT 1 IN (1, 2, 3);
SELECT toUInt16(1) IN (1, 1000, 3);
SELECT [1, 2, 3] IN (3, 4, 5);
SELECT materialize([1, 2, 3]) IN (3, 4, 5);
SELECT 'Hello' IN ('Hello', 'world');
SELECT (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) IN ((1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17));
SELECT (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, '') IN ((1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, ''));
SELECT (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, '') IN (1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 'a');
SELECT (number AS n, n + 1, n + 2, n + 3) IN (1, 2, 3, 4) FROM system.numbers LIMIT 3;
SELECT (number AS n, n + 1, n + 2, n + 3, n - 1) IN (1, 2, 3, 4, 0) FROM system.numbers LIMIT 3;
SELECT (number AS n, n + 1, toString(n + 2), n + 3, n - 1) IN (1, 2, '3', 4, 0) FROM system.numbers LIMIT 3;
SELECT [1, 2, 3] IN (2);
SELECT [1, 2, 3] IN (4);
SELECT [1, 2, 3] NOT IN (1);
SELECT [1, 2, 3] NOT IN (1, 2);
SELECT [1, 2, 3] NOT IN (1, 2, 3);
SELECT [1, 2, 3] NOT IN (1, 2, 3, 4);
SELECT ['Hello', 'world'] IN ('world');
SELECT ['Hello', 'world'] NOT IN ('world');
SELECT ['Hello', 'world'] NOT IN ('Hello', 'world');
SELECT ['Hello', 'world'] NOT IN ('hello', 'world');