This commit is contained in:
Evgeniy Gatov 2015-03-23 20:23:14 +03:00
commit b72a2f5c64
102 changed files with 7220 additions and 957 deletions

View File

@ -0,0 +1,90 @@
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
namespace DB
{
/** Сделано в целях отладки. Подлежит удалению.
*/
struct AggregateFunctionDebugData
{
UInt32 value;
AggregateFunctionDebugData()
{
value = 0xAAAAAAAA;
if (rand() % 1000 == 0)
throw Exception("Test1");
}
~AggregateFunctionDebugData()
{
try
{
if (value == 0xDEADDEAD)
throw Exception("Double free");
if (value != 0xAAAAAAAA)
throw Exception("Corruption");
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
std::terminate();
}
value = 0xDEADDEAD;
}
};
class AggregateFunctionDebug final : public IUnaryAggregateFunction<AggregateFunctionDebugData, AggregateFunctionDebug>
{
public:
String getName() const { return "debug"; }
DataTypePtr getReturnType() const
{
return new DataTypeUInt32;
}
void setArgument(const DataTypePtr & argument)
{
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
if (rand() % 1000 == 0)
throw Exception("Test2");
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
{
if (rand() % 1000 == 0)
throw Exception("Test3");
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
{
if (rand() % 1000 == 0)
throw Exception("Test4");
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
{
if (rand() % 1000 == 0)
throw Exception("Test5");
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
{
if (rand() % 1000 == 0)
throw Exception("Test6");
static_cast<ColumnUInt32 &>(to).getData().push_back(123);
}
};
}

View File

@ -147,10 +147,10 @@ struct __attribute__((__packed__)) SingleValueDataString
static constexpr Int32 AUTOMATIC_STORAGE_SIZE = 64;
static constexpr Int32 MAX_SMALL_STRING_SIZE = AUTOMATIC_STORAGE_SIZE - sizeof(size);
union
union __attribute__((__aligned__(1)))
{
char small_data[MAX_SMALL_STRING_SIZE]; /// Включая завершающий ноль.
char * large_data;
char * __attribute__((__aligned__(1))) large_data;
};
~SingleValueDataString()

View File

@ -55,13 +55,15 @@ public:
Protocol::Compression::Enum compression_ = Protocol::Compression::Enable,
Poco::Timespan connect_timeout_ = Poco::Timespan(DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, 0),
Poco::Timespan receive_timeout_ = Poco::Timespan(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, 0),
Poco::Timespan send_timeout_ = Poco::Timespan(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0))
Poco::Timespan send_timeout_ = Poco::Timespan(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0),
Poco::Timespan ping_timeout_ = Poco::Timespan(DBMS_DEFAULT_PING_TIMEOUT_SEC, 0))
:
host(host_), port(port_), default_database(default_database_),
user(user_), password(password_),
client_name(client_name_),
compression(compression_), data_type_factory(data_type_factory_),
connect_timeout(connect_timeout_), receive_timeout(receive_timeout_), send_timeout(send_timeout_),
ping_timeout(ping_timeout_),
log_wrapper(host, port)
{
/// Соединеняемся не сразу, а при первой необходимости.
@ -178,6 +180,7 @@ private:
Poco::Timespan connect_timeout;
Poco::Timespan receive_timeout;
Poco::Timespan send_timeout;
Poco::Timespan ping_timeout;
/// Откуда читать результат выполнения запроса.
SharedPtr<ReadBuffer> maybe_compressed_in;

View File

@ -91,6 +91,8 @@ private:
void applyLoadBalancing(Settings * settings)
{
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
LoadBalancing load_balancing = default_load_balancing;
if (settings)
load_balancing = settings->load_balancing;

View File

@ -18,10 +18,13 @@ public:
typedef PODArray<UInt8> Chars_t;
private:
/// Байты строк, уложенные подряд. Строки хранятся без завершающего нулевого байта.
/** NOTE Требуется, чтобы смещение и тип chars в объекте был таким же, как у data в ColumnVector<UInt8>.
* Это используется в функции packFixed (AggregationCommon.h)
*/
Chars_t chars;
/// Размер строк.
const size_t n;
/// Байты строк, уложенные подряд. Строки хранятся без завершающего нулевого байта.
Chars_t chars;
public:
/** Создать пустой столбец строк фиксированной длины n */

View File

@ -583,7 +583,21 @@ protected:
if (unlikely(grower.overflow(m_size)))
{
resize();
try
{
resize();
}
catch (...)
{
/** Если этого не делать, то будут проблемы.
* Ведь останется ключ, но неинициализированное mapped-значение,
* у которого, возможно, даже нельзя вызвать деструктор.
*/
--m_size;
buf[place_value].setZero();
throw;
}
it = find(x, hash_value);
}
}

View File

@ -52,7 +52,7 @@ private:
char * c_end;
char * c_end_of_storage;
bool use_libc_realloc;
bool use_libc_realloc = false;
T * t_start() { return reinterpret_cast<T *>(c_start); }
T * t_end() { return reinterpret_cast<T *>(c_end); }
@ -131,18 +131,22 @@ private:
if (use_libc_realloc)
{
c_start = reinterpret_cast<char *>(::realloc(c_start, bytes_to_alloc));
auto new_c_start = reinterpret_cast<char *>(::realloc(c_start, bytes_to_alloc));
if (nullptr == c_start)
if (nullptr == new_c_start)
throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
c_start = new_c_start;
}
else
{
c_start = reinterpret_cast<char *>(malloc(bytes_to_alloc));
auto new_c_start = reinterpret_cast<char *>(malloc(bytes_to_alloc));
if (nullptr == c_start)
if (nullptr == new_c_start)
throwFromErrno("PODArray: cannot realloc", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
c_start = new_c_start;
memcpy(c_start, old_c_start, std::min(bytes_to_alloc, static_cast<size_t>(end_diff)));
Allocator::deallocate(old_c_start, old_c_end_of_storage - old_c_start);
}
@ -171,10 +175,10 @@ public:
};
PODArray() : use_libc_realloc(false) { alloc(0); }
PODArray(size_t n) : use_libc_realloc(false) { alloc(n); c_end += byte_size(n); }
PODArray(size_t n, const T & x) : use_libc_realloc(false) { alloc(n); assign(n, x); }
PODArray(const_iterator from_begin, const_iterator from_end) : use_libc_realloc(false) { alloc(from_end - from_begin); insert(from_begin, from_end); }
PODArray() { alloc(0); }
PODArray(size_t n) { alloc(n); c_end += byte_size(n); }
PODArray(size_t n, const T & x) { alloc(n); assign(n, x); }
PODArray(const_iterator from_begin, const_iterator from_end) { alloc(from_end - from_begin); insert(from_begin, from_end); }
~PODArray() { dealloc(); }
PODArray(PODArray && other) { *this = std::move(other); }

View File

@ -11,6 +11,7 @@
#define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS 50
#define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300
#define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300
#define DBMS_DEFAULT_PING_TIMEOUT_SEC 5
#define DBMS_DEFAULT_POLL_INTERVAL 10
/// Насколько секунд можно максимально задерживать вставку в таблицу типа MergeTree, если в ней много недомердженных кусков.
@ -46,7 +47,9 @@
#define DEFAULT_INTERACTIVE_DELAY 100000
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD 300 /// каждый период уменьшаем счетчик ошибок в 2 раза
/// каждый период уменьшаем счетчик ошибок в 2 раза
/// слишком маленький период может приводить, что ошибки исчезают сразу после создания.
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD (2*DBMS_DEFAULT_SEND_TIMEOUT_SEC)
#define DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS 5000 /// Максимальное время ожидания в очереди запросов.
#define DBMS_DEFAULT_BACKGROUND_POOL_SIZE 6
@ -68,5 +71,8 @@
#define DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS 100
/// Граница, на которых должны быть выровнены блоки для асинхронных файловых операций.
#define DEFAULT_AIO_FILE_BLOCK_SIZE 512
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))

View File

@ -277,6 +277,14 @@ namespace ErrorCodes
STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS,
CPUID_ERROR,
INFINITE_LOOP,
CANNOT_COMPRESS,
CANNOT_DECOMPRESS,
AIO_SUBMIT_ERROR,
AIO_COMPLETION_ERROR,
AIO_READ_ERROR,
AIO_WRITE_ERROR,
AIO_UNALIGNED_SIZE_ERROR,
INDEX_NOT_USED,
POCO_EXCEPTION = 1000,
STD_EXCEPTION,

View File

@ -18,16 +18,6 @@ using Poco::SharedPtr;
class AggregatingBlockInputStream : public IProfilingBlockInputStream
{
public:
AggregatingBlockInputStream(BlockInputStreamPtr input_, const ColumnNumbers & keys_, AggregateDescriptions & aggregates_,
bool overflow_row_, bool final_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_,
Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_)
: aggregator(keys_, aggregates_, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_,
compiler_, min_count_to_compile_, group_by_two_level_threshold_),
final(final_)
{
children.push_back(input_);
}
/** keys берутся из GROUP BY части запроса
* Агрегатные функции ищутся везде в выражении.
* Столбцы, соответствующие keys и аргументам агрегатных функций, уже должны быть вычислены.

View File

@ -15,10 +15,8 @@ class CollapsingFinalBlockInputStream : public IProfilingBlockInputStream
{
public:
CollapsingFinalBlockInputStream(BlockInputStreams inputs_, const SortDescription & description_,
const String & sign_column_)
: description(description_), sign_column(sign_column_),
log(&Logger::get("CollapsingFinalBlockInputStream")),
first(true), count_positive(0), count_negative(0), count_incorrect_data(0), blocks_fetched(0), blocks_output(0)
const String & sign_column_name_)
: description(description_), sign_column_name(sign_column_name_)
{
children.insert(children.end(), inputs_.begin(), inputs_.end());
}
@ -40,7 +38,7 @@ public:
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ", sign_column, " << sign_column << ")";
res << ", sign_column, " << sign_column_name << ")";
return res.str();
}
@ -55,10 +53,10 @@ private:
{
MergingBlock(Block block_,
size_t stream_index_,
SortDescription desc_,
const SortDescription & desc,
String sign_column_name,
BlockPlainPtrs * output_blocks)
: block(block_), stream_index(stream_index_), desc(desc_), refcount(0), output_blocks(output_blocks)
: block(block_), stream_index(stream_index_), output_blocks(output_blocks)
{
sort_columns.resize(desc.size());
for (size_t i = 0; i < desc.size(); ++i)
@ -86,8 +84,6 @@ private:
/// Строки с одинаковым ключом будут упорядочены по возрастанию stream_index.
size_t stream_index;
SortDescription desc;
size_t rows;
/// Какие строки нужно оставить. Заполняется при слиянии потоков.
@ -98,7 +94,7 @@ private:
const ColumnInt8 * sign_column;
/// Когда достигает нуля, блок можно выдавать в ответ.
int refcount;
int refcount = 0;
/// Куда положить блок, когда он готов попасть в ответ.
BlockPlainPtrs * output_blocks;
@ -181,17 +177,17 @@ private:
Cursor() {}
explicit Cursor(MergingBlockPtr block_, size_t pos_ = 0) : block(block_), pos(pos_) {}
bool operator<(const Cursor & rhs) const
bool operator< (const Cursor & rhs) const
{
for (size_t i = 0; i < block->sort_columns.size(); ++i)
{
int direction = block->desc[i].direction;
int res = direction * block->sort_columns[i]->compareAt(pos, rhs.pos, *(rhs.block->sort_columns[i]), direction);
int res = block->sort_columns[i]->compareAt(pos, rhs.pos, *(rhs.block->sort_columns[i]), 1);
if (res > 0)
return true;
if (res < 0)
return false;
}
return block->stream_index > rhs.block->stream_index;
}
@ -203,7 +199,7 @@ private:
for (size_t i = 0; i < block->sort_columns.size(); ++i)
{
int res = block->desc[i].direction * block->sort_columns[i]->compareAt(pos, rhs.pos, *(rhs.block->sort_columns[i]), 1);
int res = block->sort_columns[i]->compareAt(pos, rhs.pos, *(rhs.block->sort_columns[i]), 1);
if (res != 0)
return false;
}
@ -235,12 +231,12 @@ private:
typedef std::priority_queue<Cursor> Queue;
SortDescription description;
String sign_column;
const SortDescription description;
String sign_column_name;
Logger * log;
Logger * log = &Logger::get("CollapsingFinalBlockInputStream");
bool first;
bool first = true;
BlockPlainPtrs output_blocks;
@ -249,15 +245,15 @@ private:
Cursor previous; /// Текущий первичный ключ.
Cursor last_positive; /// Последняя положительная строка для текущего первичного ключа.
size_t count_positive; /// Количество положительных строк для текущего первичного ключа.
size_t count_negative; /// Количество отрицательных строк для текущего первичного ключа.
bool last_is_positive; /// true, если последняя строка для текущего первичного ключа положительная.
size_t count_positive = 0; /// Количество положительных строк для текущего первичного ключа.
size_t count_negative = 0; /// Количество отрицательных строк для текущего первичного ключа.
bool last_is_positive = false; /// true, если последняя строка для текущего первичного ключа положительная.
size_t count_incorrect_data; /// Чтобы не писать в лог слишком много сообщений об ошибке.
size_t count_incorrect_data = 0; /// Чтобы не писать в лог слишком много сообщений об ошибке.
/// Посчитаем, сколько блоков получили на вход и отдали на выход.
size_t blocks_fetched;
size_t blocks_output;
size_t blocks_fetched = 0;
size_t blocks_output = 0;
void fetchNextBlock(size_t input_index);
void commitCurrent();

View File

@ -7,9 +7,9 @@
#include <DB/Interpreters/ProcessList.h>
#include <DB/DataStreams/BlockStreamProfileInfo.h>
#include <DB/DataStreams/IBlockInputStream.h>
#include <atomic>
namespace DB
{
@ -96,9 +96,9 @@ public:
/** Требуется ли прервать получение данных.
*/
bool isCancelled()
bool isCancelled() const
{
return is_cancelled;
return is_cancelled.load(std::memory_order_seq_cst);
}
/** Какие ограничения (и квоты) проверяются.
@ -117,27 +117,19 @@ public:
/// Используется подмножество ограничений из Limits.
struct LocalLimits
{
LimitsMode mode;
LimitsMode mode = LIMITS_CURRENT;
size_t max_rows_to_read;
size_t max_bytes_to_read;
OverflowMode read_overflow_mode;
size_t max_rows_to_read = 0;
size_t max_bytes_to_read = 0;
OverflowMode read_overflow_mode = OverflowMode::THROW;
Poco::Timespan max_execution_time;
OverflowMode timeout_overflow_mode;
Poco::Timespan max_execution_time = 0;
OverflowMode timeout_overflow_mode = OverflowMode::THROW;
/// В строчках в секунду.
size_t min_execution_speed;
size_t min_execution_speed = 0;
/// Проверять, что скорость не слишком низкая, после прошествия указанного времени.
Poco::Timespan timeout_before_checking_execution_speed;
LocalLimits()
: mode(LIMITS_CURRENT),
max_rows_to_read(0), max_bytes_to_read(0), read_overflow_mode(OverflowMode::THROW),
max_execution_time(0), timeout_overflow_mode(OverflowMode::THROW),
min_execution_speed(0), timeout_before_checking_execution_speed(0)
{
}
Poco::Timespan timeout_before_checking_execution_speed = 0;
};
/** Установить ограничения для проверки на каждый блок. */
@ -159,7 +151,7 @@ public:
protected:
BlockStreamProfileInfo info;
volatile bool is_cancelled = false;
std::atomic<bool> is_cancelled{false};
ProgressCallback progress_callback;
ProcessList::Element * process_list_elem = nullptr;

View File

@ -16,14 +16,6 @@ using Poco::SharedPtr;
class MergingAggregatedBlockInputStream : public IProfilingBlockInputStream
{
public:
MergingAggregatedBlockInputStream(BlockInputStreamPtr input_, const ColumnNumbers & keys_,
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_)
: aggregator(keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0),
final(final_), max_threads(max_threads_)
{
children.push_back(input_);
}
MergingAggregatedBlockInputStream(BlockInputStreamPtr input_, const Names & keys_names_,
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_)
: aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0),

View File

@ -19,19 +19,6 @@ using Poco::SharedPtr;
class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream
{
public:
ParallelAggregatingBlockInputStream(BlockInputStreams inputs, const ColumnNumbers & keys_,
AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_,
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_,
Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_)
: aggregator(keys_, aggregates_, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_,
compiler_, min_count_to_compile_, group_by_two_level_threshold_),
final(final_), max_threads(std::min(inputs.size(), max_threads_)),
keys_size(keys_.size()), aggregates_size(aggregates_.size()),
handler(*this), processor(inputs, max_threads, handler)
{
children.insert(children.end(), inputs.begin(), inputs.end());
}
/** Столбцы из key_names и аргументы агрегатных функций, уже должны быть вычислены.
*/
ParallelAggregatingBlockInputStream(BlockInputStreams inputs, const Names & key_names,
@ -41,7 +28,7 @@ public:
: aggregator(key_names, aggregates, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_,
compiler_, min_count_to_compile_, group_by_two_level_threshold_),
final(final_), max_threads(std::min(inputs.size(), max_threads_)),
keys_size(key_names.size()), aggregates_size(aggregates.size()),
keys_size(aggregator.getNumberOfKeys()), aggregates_size(aggregator.getNumberOfAggregates()),
handler(*this), processor(inputs, max_threads, handler)
{
children.insert(children.end(), inputs.begin(), inputs.end());
@ -70,7 +57,8 @@ public:
void cancel() override
{
if (!__sync_bool_compare_and_swap(&is_cancelled, false, true))
bool old_val = false;
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
processor.cancel();
@ -122,6 +110,31 @@ private:
Logger * log = &Logger::get("ParallelAggregatingBlockInputStream");
ManyAggregatedDataVariants many_data;
Exceptions exceptions;
struct ThreadData
{
size_t src_rows = 0;
size_t src_bytes = 0;
StringRefs key;
ConstColumnPlainPtrs key_columns;
Aggregator::AggregateColumns aggregate_columns;
Sizes key_sizes;
ThreadData(size_t keys_size, size_t aggregates_size)
{
key.resize(keys_size);
key_columns.resize(keys_size);
aggregate_columns.resize(aggregates_size);
key_sizes.resize(keys_size);
}
};
std::vector<ThreadData> threads_data;
struct Handler
{
Handler(ParallelAggregatingBlockInputStream & parent_)
@ -154,29 +167,6 @@ private:
Handler handler;
ParallelInputsProcessor<Handler> processor;
ManyAggregatedDataVariants many_data;
Exceptions exceptions;
struct ThreadData
{
size_t src_rows = 0;
size_t src_bytes = 0;
StringRefs key;
ConstColumnPlainPtrs key_columns;
Aggregator::AggregateColumns aggregate_columns;
Sizes key_sizes;
ThreadData(size_t keys_size, size_t aggregates_size)
{
key.resize(keys_size);
key_columns.resize(keys_size);
aggregate_columns.resize(aggregates_size);
key_sizes.resize(keys_size);
}
};
std::vector<ThreadData> threads_data;
AggregatedDataVariantsPtr executeAndMerge()
{

View File

@ -80,7 +80,8 @@ public:
void cancel() override
{
if (!__sync_bool_compare_and_swap(&is_cancelled, false, true))
bool old_val = false;
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
if (hasNoQueryInProgress() || hasThrownException())
@ -147,6 +148,9 @@ protected:
while (true)
{
if (isCancelled())
return Block();
Connection::Packet packet = parallel_replicas->receivePacket();
switch (packet.type)
@ -178,10 +182,6 @@ protected:
* и квот (например, на количество строчек для чтения).
*/
progressImpl(packet.progress);
if (isQueryInProgress() && isCancelled())
cancel();
break;
case Protocol::Server::ProfileInfo:
@ -281,7 +281,7 @@ private:
void tryCancel(const char * reason)
{
bool old_val = false;
if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_seq_cst))
if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
LOG_TRACE(log, "(" << parallel_replicas->dumpAddresses() << ") " << reason);

View File

@ -76,7 +76,8 @@ public:
*/
void cancel() override
{
if (!__sync_bool_compare_and_swap(&is_cancelled, false, true))
bool old_val = false;
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
//std::cerr << "cancelling\n";
@ -164,7 +165,7 @@ protected:
void readSuffix() override
{
//std::cerr << "readSuffix\n";
if (!all_read && !is_cancelled)
if (!all_read && !is_cancelled.load(std::memory_order_seq_cst))
throw Exception("readSuffix called before all data is read", ErrorCodes::LOGICAL_ERROR);
finalize();

View File

@ -6,6 +6,7 @@
#include <DB/DataStreams/IRowInputStream.h>
#include <DB/DataStreams/IRowOutputStream.h>
#include <atomic>
namespace DB
{
@ -13,7 +14,7 @@ namespace DB
/** Копирует данные из InputStream в OutputStream
* (например, из БД в консоль и т. п.)
*/
void copyData(IBlockInputStream & from, IBlockOutputStream & to, volatile bool * is_cancelled = nullptr);
void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic<bool> * is_cancelled = nullptr);
void copyData(IRowInputStream & from, IRowOutputStream & to);
}

View File

@ -2,6 +2,8 @@
#include <map>
#include <Yandex/singleton.h>
#include <Poco/RegularExpression.h>
#include <DB/DataTypes/IDataType.h>
@ -12,7 +14,7 @@ namespace DB
/** Позволяет создать тип данных по его имени.
*/
class DataTypeFactory
class DataTypeFactory : public Singleton<DataTypeFactory>
{
public:
DataTypeFactory();

View File

@ -68,11 +68,11 @@ public:
getItems<UInt64>(*hierarchical_attribute, ids, out);
}
#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \
#define DECLARE_INDIVIDUAL_GETTER(TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
@ -83,21 +83,21 @@ public:
getItems<TYPE>(attribute, ids, out);\
return out.front();\
}
DECLARE_INDIVIDUAL_GETTER(UInt8, uint8)
DECLARE_INDIVIDUAL_GETTER(UInt16, uint16)
DECLARE_INDIVIDUAL_GETTER(UInt32, uint32)
DECLARE_INDIVIDUAL_GETTER(UInt64, uint64)
DECLARE_INDIVIDUAL_GETTER(Int8, int8)
DECLARE_INDIVIDUAL_GETTER(Int16, int16)
DECLARE_INDIVIDUAL_GETTER(Int32, int32)
DECLARE_INDIVIDUAL_GETTER(Int64, int64)
DECLARE_INDIVIDUAL_GETTER(Float32, float32)
DECLARE_INDIVIDUAL_GETTER(Float64, float64)
DECLARE_INDIVIDUAL_GETTER(UInt8)
DECLARE_INDIVIDUAL_GETTER(UInt16)
DECLARE_INDIVIDUAL_GETTER(UInt32)
DECLARE_INDIVIDUAL_GETTER(UInt64)
DECLARE_INDIVIDUAL_GETTER(Int8)
DECLARE_INDIVIDUAL_GETTER(Int16)
DECLARE_INDIVIDUAL_GETTER(Int32)
DECLARE_INDIVIDUAL_GETTER(Int64)
DECLARE_INDIVIDUAL_GETTER(Float32)
DECLARE_INDIVIDUAL_GETTER(Float64)
#undef DECLARE_INDIVIDUAL_GETTER
String getString(const std::string & attribute_name, const id_t id) const override
{
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
@ -110,11 +110,11 @@ public:
return String{out.getDataAt(0)};
};
#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
{\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
@ -122,21 +122,21 @@ public:
\
getItems<TYPE>(attribute, ids, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8, uint8)
DECLARE_MULTIPLE_GETTER(UInt16, uint16)
DECLARE_MULTIPLE_GETTER(UInt32, uint32)
DECLARE_MULTIPLE_GETTER(UInt64, uint64)
DECLARE_MULTIPLE_GETTER(Int8, int8)
DECLARE_MULTIPLE_GETTER(Int16, int16)
DECLARE_MULTIPLE_GETTER(Int32, int32)
DECLARE_MULTIPLE_GETTER(Int64, int64)
DECLARE_MULTIPLE_GETTER(Float32, float32)
DECLARE_MULTIPLE_GETTER(Float64, float64)
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
DECLARE_MULTIPLE_GETTER(UInt32)
DECLARE_MULTIPLE_GETTER(UInt64)
DECLARE_MULTIPLE_GETTER(Int8)
DECLARE_MULTIPLE_GETTER(Int16)
DECLARE_MULTIPLE_GETTER(Int32)
DECLARE_MULTIPLE_GETTER(Int64)
DECLARE_MULTIPLE_GETTER(Float32)
DECLARE_MULTIPLE_GETTER(Float64)
#undef DECLARE_MULTIPLE_GETTER
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const override
{
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
@ -154,7 +154,7 @@ private:
struct attribute_t final
{
AttributeType type;
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
@ -180,14 +180,13 @@ private:
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attributes.push_back(createAttributeWithType(getAttributeTypeByName(attribute.type),
attribute.null_value));
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
{
hierarchical_attribute = &attributes.back();
if (hierarchical_attribute->type != AttributeType::uint64)
if (hierarchical_attribute->type != AttributeUnderlyingType::UInt64)
throw Exception{
"Hierarchical attribute must be UInt64.",
ErrorCodes::TYPE_MISMATCH
@ -196,54 +195,54 @@ private:
}
}
attribute_t createAttributeWithType(const AttributeType type, const std::string & null_value)
attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
{
attribute_t attr{type};
switch (type)
{
case AttributeType::uint8:
std::get<UInt8>(attr.null_values) = DB::parse<UInt8>(null_value);
case AttributeUnderlyingType::UInt8:
std::get<UInt8>(attr.null_values) = null_value.get<UInt64>();
std::get<std::unique_ptr<UInt8[]>>(attr.arrays) = std::make_unique<UInt8[]>(size);
break;
case AttributeType::uint16:
std::get<UInt16>(attr.null_values) = DB::parse<UInt16>(null_value);
case AttributeUnderlyingType::UInt16:
std::get<UInt16>(attr.null_values) = null_value.get<UInt64>();
std::get<std::unique_ptr<UInt16[]>>(attr.arrays) = std::make_unique<UInt16[]>(size);
break;
case AttributeType::uint32:
std::get<UInt32>(attr.null_values) = DB::parse<UInt32>(null_value);
case AttributeUnderlyingType::UInt32:
std::get<UInt32>(attr.null_values) = null_value.get<UInt64>();
std::get<std::unique_ptr<UInt32[]>>(attr.arrays) = std::make_unique<UInt32[]>(size);
break;
case AttributeType::uint64:
std::get<UInt64>(attr.null_values) = DB::parse<UInt64>(null_value);
case AttributeUnderlyingType::UInt64:
std::get<UInt64>(attr.null_values) = null_value.get<UInt64>();
std::get<std::unique_ptr<UInt64[]>>(attr.arrays) = std::make_unique<UInt64[]>(size);
break;
case AttributeType::int8:
std::get<Int8>(attr.null_values) = DB::parse<Int8>(null_value);
case AttributeUnderlyingType::Int8:
std::get<Int8>(attr.null_values) = null_value.get<Int64>();
std::get<std::unique_ptr<Int8[]>>(attr.arrays) = std::make_unique<Int8[]>(size);
break;
case AttributeType::int16:
std::get<Int16>(attr.null_values) = DB::parse<Int16>(null_value);
case AttributeUnderlyingType::Int16:
std::get<Int16>(attr.null_values) = null_value.get<Int64>();
std::get<std::unique_ptr<Int16[]>>(attr.arrays) = std::make_unique<Int16[]>(size);
break;
case AttributeType::int32:
std::get<Int32>(attr.null_values) = DB::parse<Int32>(null_value);
case AttributeUnderlyingType::Int32:
std::get<Int32>(attr.null_values) = null_value.get<Int64>();
std::get<std::unique_ptr<Int32[]>>(attr.arrays) = std::make_unique<Int32[]>(size);
break;
case AttributeType::int64:
std::get<Int64>(attr.null_values) = DB::parse<Int64>(null_value);
case AttributeUnderlyingType::Int64:
std::get<Int64>(attr.null_values) = null_value.get<Int64>();
std::get<std::unique_ptr<Int64[]>>(attr.arrays) = std::make_unique<Int64[]>(size);
break;
case AttributeType::float32:
std::get<Float32>(attr.null_values) = DB::parse<Float32>(null_value);
case AttributeUnderlyingType::Float32:
std::get<Float32>(attr.null_values) = null_value.get<Float64>();
std::get<std::unique_ptr<Float32[]>>(attr.arrays) = std::make_unique<Float32[]>(size);
break;
case AttributeType::float64:
std::get<Float64>(attr.null_values) = DB::parse<Float64>(null_value);
case AttributeUnderlyingType::Float64:
std::get<Float64>(attr.null_values) = null_value.get<Float64>();
std::get<std::unique_ptr<Float64[]>>(attr.arrays) = std::make_unique<Float64[]>(size);
break;
case AttributeType::string:
std::get<String>(attr.null_values) = null_value;
case AttributeUnderlyingType::String:
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<std::unique_ptr<StringRef[]>>(attr.arrays) = std::make_unique<StringRef[]>(size);
break;
}
@ -495,17 +494,17 @@ private:
{
switch (attribute.type)
{
case AttributeType::uint8: std::get<std::unique_ptr<UInt8[]>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values); break;
case AttributeType::uint16: std::get<std::unique_ptr<UInt16[]>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values); break;
case AttributeType::uint32: std::get<std::unique_ptr<UInt32[]>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values); break;
case AttributeType::uint64: std::get<std::unique_ptr<UInt64[]>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values); break;
case AttributeType::int8: std::get<std::unique_ptr<Int8[]>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values); break;
case AttributeType::int16: std::get<std::unique_ptr<Int16[]>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values); break;
case AttributeType::int32: std::get<std::unique_ptr<Int32[]>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values); break;
case AttributeType::int64: std::get<std::unique_ptr<Int64[]>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values); break;
case AttributeType::float32: std::get<std::unique_ptr<Float32[]>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values); break;
case AttributeType::float64: std::get<std::unique_ptr<Float64[]>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values); break;
case AttributeType::string:
case AttributeUnderlyingType::UInt8: std::get<std::unique_ptr<UInt8[]>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values); break;
case AttributeUnderlyingType::UInt16: std::get<std::unique_ptr<UInt16[]>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values); break;
case AttributeUnderlyingType::UInt32: std::get<std::unique_ptr<UInt32[]>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values); break;
case AttributeUnderlyingType::UInt64: std::get<std::unique_ptr<UInt64[]>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values); break;
case AttributeUnderlyingType::Int8: std::get<std::unique_ptr<Int8[]>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values); break;
case AttributeUnderlyingType::Int16: std::get<std::unique_ptr<Int16[]>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values); break;
case AttributeUnderlyingType::Int32: std::get<std::unique_ptr<Int32[]>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values); break;
case AttributeUnderlyingType::Int64: std::get<std::unique_ptr<Int64[]>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values); break;
case AttributeUnderlyingType::Float32: std::get<std::unique_ptr<Float32[]>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values); break;
case AttributeUnderlyingType::Float64: std::get<std::unique_ptr<Float64[]>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values); break;
case AttributeUnderlyingType::String:
{
const auto & null_value_ref = std::get<String>(attribute.null_values);
auto & string_ref = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays)[idx];
@ -525,17 +524,17 @@ private:
{
switch (attribute.type)
{
case AttributeType::uint8: std::get<std::unique_ptr<UInt8[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeType::uint16: std::get<std::unique_ptr<UInt16[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeType::uint32: std::get<std::unique_ptr<UInt32[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeType::uint64: std::get<std::unique_ptr<UInt64[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeType::int8: std::get<std::unique_ptr<Int8[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeType::int16: std::get<std::unique_ptr<Int16[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeType::int32: std::get<std::unique_ptr<Int32[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeType::int64: std::get<std::unique_ptr<Int64[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeType::float32: std::get<std::unique_ptr<Float32[]>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeType::float64: std::get<std::unique_ptr<Float64[]>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeType::string:
case AttributeUnderlyingType::UInt8: std::get<std::unique_ptr<UInt8[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt16: std::get<std::unique_ptr<UInt16[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt32: std::get<std::unique_ptr<UInt32[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt64: std::get<std::unique_ptr<UInt64[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::Int8: std::get<std::unique_ptr<Int8[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int16: std::get<std::unique_ptr<Int16[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int32: std::get<std::unique_ptr<Int32[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int64: std::get<std::unique_ptr<Int64[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Float32: std::get<std::unique_ptr<Float32[]>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::Float64: std::get<std::unique_ptr<Float64[]>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::String:
{
const auto & string = value.get<String>();
auto & string_ref = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays)[idx];

View File

@ -35,13 +35,13 @@ public:
max_connections, host, port, db, user, password, context.getDataTypeFactory(),
"ClickHouseDictionarySource")
},
load_all_query{composeLoadAllQuery(sample_block, table)}
load_all_query{composeLoadAllQuery(sample_block, db, table)}
{}
/// copy-constructor is provided in order to support cloneability
ClickHouseDictionarySource(const ClickHouseDictionarySource & other)
: host{other.host}, port{other.port}, user{other.user}, password{other.password},
db{other.db}, table{other.db},
db{other.db}, table{other.table},
sample_block{other.sample_block}, context(other.context),
is_local{other.is_local},
pool{is_local ? nullptr : std::make_unique<ConnectionPool>(
@ -75,7 +75,7 @@ public:
DictionarySourcePtr clone() const override { return std::make_unique<ClickHouseDictionarySource>(*this); }
private:
static std::string composeLoadAllQuery(const Block & block, const std::string & table)
static std::string composeLoadAllQuery(const Block & block, const std::string & db, const std::string & table)
{
std::string query;
@ -94,6 +94,8 @@ private:
}
writeString(" FROM ", out);
writeProbablyBackQuotedString(db, out);
writeChar('.', out);
writeProbablyBackQuotedString(table, out);
writeChar(';', out);
}
@ -121,6 +123,8 @@ private:
const auto & id_column_name = sample_block.getByPosition(0).name;
writeString(" FROM ", out);
writeProbablyBackQuotedString(db, out);
writeChar('.', out);
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
writeProbablyBackQuotedString(id_column_name, out);

View File

@ -15,7 +15,7 @@ namespace DB
namespace
{
Block createSampleBlock(const DictionaryStructure & dict_struct, const Context & context)
Block createSampleBlock(const DictionaryStructure & dict_struct)
{
Block block{
ColumnWithNameAndType{
@ -26,12 +26,9 @@ Block createSampleBlock(const DictionaryStructure & dict_struct, const Context &
};
for (const auto & attribute : dict_struct.attributes)
{
const auto & type = context.getDataTypeFactory().get(attribute.type);
block.insert(ColumnWithNameAndType{
type->createColumn(), type, attribute.name
attribute.type->createColumn(), attribute.type, attribute.name
});
}
return block;
}
@ -55,7 +52,7 @@ public:
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG
};
auto sample_block = createSampleBlock(dict_struct, context);
auto sample_block = createSampleBlock(dict_struct);
const auto & source_type = keys.front();

View File

@ -1,6 +1,8 @@
#pragma once
#include <DB/Core/ErrorCodes.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/IO/ReadBufferFromString.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <vector>
#include <string>
@ -9,35 +11,37 @@
namespace DB
{
enum class AttributeType
enum class AttributeUnderlyingType
{
uint8,
uint16,
uint32,
uint64,
int8,
int16,
int32,
int64,
float32,
float64,
string
UInt8,
UInt16,
UInt32,
UInt64,
Int8,
Int16,
Int32,
Int64,
Float32,
Float64,
String
};
inline AttributeType getAttributeTypeByName(const std::string & type)
inline AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type)
{
static const std::unordered_map<std::string, AttributeType> dictionary{
{ "UInt8", AttributeType::uint8 },
{ "UInt16", AttributeType::uint16 },
{ "UInt32", AttributeType::uint32 },
{ "UInt64", AttributeType::uint64 },
{ "Int8", AttributeType::int8 },
{ "Int16", AttributeType::int16 },
{ "Int32", AttributeType::int32 },
{ "Int64", AttributeType::int64 },
{ "Float32", AttributeType::float32 },
{ "Float64", AttributeType::float64 },
{ "String", AttributeType::string },
static const std::unordered_map<std::string, AttributeUnderlyingType> dictionary{
{ "UInt8", AttributeUnderlyingType::UInt8 },
{ "UInt16", AttributeUnderlyingType::UInt16 },
{ "UInt32", AttributeUnderlyingType::UInt32 },
{ "UInt64", AttributeUnderlyingType::UInt64 },
{ "Int8", AttributeUnderlyingType::Int8 },
{ "Int16", AttributeUnderlyingType::Int16 },
{ "Int32", AttributeUnderlyingType::Int32 },
{ "Int64", AttributeUnderlyingType::Int64 },
{ "Float32", AttributeUnderlyingType::Float32 },
{ "Float64", AttributeUnderlyingType::Float64 },
{ "String", AttributeUnderlyingType::String },
{ "Date", AttributeUnderlyingType::UInt16 },
{ "DateTime", AttributeUnderlyingType::UInt32 },
};
const auto it = dictionary.find(type);
@ -50,21 +54,21 @@ inline AttributeType getAttributeTypeByName(const std::string & type)
};
}
inline std::string toString(const AttributeType type)
inline std::string toString(const AttributeUnderlyingType type)
{
switch (type)
{
case AttributeType::uint8: return "UInt8";
case AttributeType::uint16: return "UInt16";
case AttributeType::uint32: return "UInt32";
case AttributeType::uint64: return "UInt64";
case AttributeType::int8: return "Int8";
case AttributeType::int16: return "Int16";
case AttributeType::int32: return "Int32";
case AttributeType::int64: return "Int64";
case AttributeType::float32: return "Float32";
case AttributeType::float64: return "Float64";
case AttributeType::string: return "String";
case AttributeUnderlyingType::UInt8: return "UInt8";
case AttributeUnderlyingType::UInt16: return "UInt16";
case AttributeUnderlyingType::UInt32: return "UInt32";
case AttributeUnderlyingType::UInt64: return "UInt64";
case AttributeUnderlyingType::Int8: return "Int8";
case AttributeUnderlyingType::Int16: return "Int16";
case AttributeUnderlyingType::Int32: return "Int32";
case AttributeUnderlyingType::Int64: return "Int64";
case AttributeUnderlyingType::Float32: return "Float32";
case AttributeUnderlyingType::Float64: return "Float64";
case AttributeUnderlyingType::String: return "String";
}
throw Exception{
@ -91,7 +95,7 @@ struct DictionaryLifetime
/** Holds the description of a single dictionary attribute:
* - name, used for lookup into dictionary and source;
* - type, used in conjunction with DataTypeFactory and getAttributeTypeByname;
* - type, used in conjunction with DataTypeFactory and getAttributeUnderlyingTypeByname;
* - null_value, used as a default value for non-existent entries in the dictionary,
* decimal representation for numeric attributes;
* - hierarchical, whether this attribute defines a hierarchy;
@ -100,8 +104,9 @@ struct DictionaryLifetime
struct DictionaryAttribute
{
std::string name;
std::string type;
std::string null_value;
AttributeUnderlyingType underlying_type;
DataTypePtr type;
Field null_value;
bool hierarchical;
bool injective;
};
@ -131,12 +136,20 @@ struct DictionaryStructure
continue;
const auto prefix = config_prefix + '.' + key + '.';
const auto name = config.getString(prefix + "name");
const auto type = config.getString(prefix + "type");
const auto null_value = config.getString(prefix + "null_value");
const auto type_string = config.getString(prefix + "type");
const auto type = DataTypeFactory::instance().get(type_string);
const auto underlying_type = getAttributeUnderlyingType(type_string);
const auto null_value_string = config.getString(prefix + "null_value");
Field null_value;
ReadBufferFromString null_value_buffer{null_value_string};
type->deserializeText(null_value, null_value_buffer);
const auto hierarchical = config.getBool(prefix + "hierarchical", false);
const auto injective = config.getBool(prefix + "injective", false);
if (name.empty() || type.empty())
if (name.empty())
throw Exception{
"Properties 'name' and 'type' of an attribute cannot be empty",
ErrorCodes::BAD_ARGUMENTS
@ -151,7 +164,7 @@ struct DictionaryStructure
has_hierarchy = has_hierarchy || hierarchical;
attributes.emplace_back(DictionaryAttribute{
name, type, null_value, hierarchical, injective
name, underlying_type, type, null_value, hierarchical, injective
});
}

View File

@ -57,11 +57,11 @@ public:
getItems<UInt64>(*hierarchical_attribute, ids, out);
}
#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \
#define DECLARE_INDIVIDUAL_GETTER(TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
@ -71,21 +71,21 @@ public:
\
return id < array.size() ? array[id] : std::get<TYPE>(attribute.null_values);\
}
DECLARE_INDIVIDUAL_GETTER(UInt8, uint8)
DECLARE_INDIVIDUAL_GETTER(UInt16, uint16)
DECLARE_INDIVIDUAL_GETTER(UInt32, uint32)
DECLARE_INDIVIDUAL_GETTER(UInt64, uint64)
DECLARE_INDIVIDUAL_GETTER(Int8, int8)
DECLARE_INDIVIDUAL_GETTER(Int16, int16)
DECLARE_INDIVIDUAL_GETTER(Int32, int32)
DECLARE_INDIVIDUAL_GETTER(Int64, int64)
DECLARE_INDIVIDUAL_GETTER(Float32, float32)
DECLARE_INDIVIDUAL_GETTER(Float64, float64)
DECLARE_INDIVIDUAL_GETTER(UInt8)
DECLARE_INDIVIDUAL_GETTER(UInt16)
DECLARE_INDIVIDUAL_GETTER(UInt32)
DECLARE_INDIVIDUAL_GETTER(UInt64)
DECLARE_INDIVIDUAL_GETTER(Int8)
DECLARE_INDIVIDUAL_GETTER(Int16)
DECLARE_INDIVIDUAL_GETTER(Int32)
DECLARE_INDIVIDUAL_GETTER(Int64)
DECLARE_INDIVIDUAL_GETTER(Float32)
DECLARE_INDIVIDUAL_GETTER(Float64)
#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)
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
@ -96,11 +96,11 @@ public:
return id < array.size() ? String{array[id]} : std::get<String>(attribute.null_values);
}
#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
@ -108,21 +108,21 @@ public:
\
getItems<TYPE>(attribute, ids, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8, uint8)
DECLARE_MULTIPLE_GETTER(UInt16, uint16)
DECLARE_MULTIPLE_GETTER(UInt32, uint32)
DECLARE_MULTIPLE_GETTER(UInt64, uint64)
DECLARE_MULTIPLE_GETTER(Int8, int8)
DECLARE_MULTIPLE_GETTER(Int16, int16)
DECLARE_MULTIPLE_GETTER(Int32, int32)
DECLARE_MULTIPLE_GETTER(Int64, int64)
DECLARE_MULTIPLE_GETTER(Float32, float32)
DECLARE_MULTIPLE_GETTER(Float64, float64)
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
DECLARE_MULTIPLE_GETTER(UInt32)
DECLARE_MULTIPLE_GETTER(UInt64)
DECLARE_MULTIPLE_GETTER(Int8)
DECLARE_MULTIPLE_GETTER(Int16)
DECLARE_MULTIPLE_GETTER(Int32)
DECLARE_MULTIPLE_GETTER(Int64)
DECLARE_MULTIPLE_GETTER(Float32)
DECLARE_MULTIPLE_GETTER(Float64)
#undef DECLARE_MULTIPLE_GETTER
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const override
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
@ -142,7 +142,7 @@ public:
private:
struct attribute_t final
{
AttributeType type;
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
@ -169,14 +169,13 @@ private:
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attributes.push_back(createAttributeWithType(getAttributeTypeByName(attribute.type),
attribute.null_value));
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
{
hierarchical_attribute = &attributes.back();
if (hierarchical_attribute->type != AttributeType::uint64)
if (hierarchical_attribute->type != AttributeUnderlyingType::UInt64)
throw Exception{
"Hierarchical attribute must be UInt64.",
ErrorCodes::TYPE_MISMATCH
@ -208,38 +207,39 @@ private:
}
template <typename T>
void createAttributeImpl(attribute_t & attribute, const std::string & null_value)
void createAttributeImpl(attribute_t & attribute, const Field & null_value)
{
const auto & null_value_ref = std::get<T>(attribute.null_values) = DB::parse<T>(null_value);
const auto & null_value_ref = std::get<T>(attribute.null_values) =
null_value.get<typename NearestFieldType<T>::Type>();
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 createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
{
attribute_t attr{type};
switch (type)
{
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:
case AttributeUnderlyingType::UInt8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::String:
{
const auto & null_value_ref = std::get<String>(attr.null_values) = DB::parse<String>(null_value);
const auto & null_value_ref = std::get<String>(attr.null_values) = null_value.get<String>();
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;
}
@ -276,17 +276,17 @@ private:
switch (attribute.type)
{
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:
case AttributeUnderlyingType::UInt8: setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt16: setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt32: setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt64: setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::Int8: setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int16: setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int32: setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int64: setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Float32: setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::Float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::String:
{
auto & array = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);
if (id >= array.size())

View File

@ -56,11 +56,11 @@ public:
getItems<UInt64>(*hierarchical_attribute, ids, out);
}
#define DECLARE_INDIVIDUAL_GETTER(TYPE, LC_TYPE) \
#define DECLARE_INDIVIDUAL_GETTER(TYPE) \
TYPE get##TYPE(const std::string & attribute_name, const id_t id) const override\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
@ -71,21 +71,21 @@ public:
\
return it != map.end() ? TYPE{it->second} : std::get<TYPE>(attribute.null_values);\
}
DECLARE_INDIVIDUAL_GETTER(UInt8, uint8)
DECLARE_INDIVIDUAL_GETTER(UInt16, uint16)
DECLARE_INDIVIDUAL_GETTER(UInt32, uint32)
DECLARE_INDIVIDUAL_GETTER(UInt64, uint64)
DECLARE_INDIVIDUAL_GETTER(Int8, int8)
DECLARE_INDIVIDUAL_GETTER(Int16, int16)
DECLARE_INDIVIDUAL_GETTER(Int32, int32)
DECLARE_INDIVIDUAL_GETTER(Int64, int64)
DECLARE_INDIVIDUAL_GETTER(Float32, float32)
DECLARE_INDIVIDUAL_GETTER(Float64, float64)
DECLARE_INDIVIDUAL_GETTER(UInt8)
DECLARE_INDIVIDUAL_GETTER(UInt16)
DECLARE_INDIVIDUAL_GETTER(UInt32)
DECLARE_INDIVIDUAL_GETTER(UInt64)
DECLARE_INDIVIDUAL_GETTER(Int8)
DECLARE_INDIVIDUAL_GETTER(Int16)
DECLARE_INDIVIDUAL_GETTER(Int32)
DECLARE_INDIVIDUAL_GETTER(Int64)
DECLARE_INDIVIDUAL_GETTER(Float32)
DECLARE_INDIVIDUAL_GETTER(Float64)
#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)
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
@ -97,11 +97,11 @@ public:
return it != map.end() ? String{it->second} : std::get<String>(attribute.null_values);
}
#define DECLARE_MULTIPLE_GETTER(TYPE, LC_TYPE)\
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeType::LC_TYPE)\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
@ -109,21 +109,21 @@ public:
\
getItems<TYPE>(attribute, ids, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8, uint8)
DECLARE_MULTIPLE_GETTER(UInt16, uint16)
DECLARE_MULTIPLE_GETTER(UInt32, uint32)
DECLARE_MULTIPLE_GETTER(UInt64, uint64)
DECLARE_MULTIPLE_GETTER(Int8, int8)
DECLARE_MULTIPLE_GETTER(Int16, int16)
DECLARE_MULTIPLE_GETTER(Int32, int32)
DECLARE_MULTIPLE_GETTER(Int64, int64)
DECLARE_MULTIPLE_GETTER(Float32, float32)
DECLARE_MULTIPLE_GETTER(Float64, float64)
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
DECLARE_MULTIPLE_GETTER(UInt32)
DECLARE_MULTIPLE_GETTER(UInt64)
DECLARE_MULTIPLE_GETTER(Int8)
DECLARE_MULTIPLE_GETTER(Int16)
DECLARE_MULTIPLE_GETTER(Int32)
DECLARE_MULTIPLE_GETTER(Int64)
DECLARE_MULTIPLE_GETTER(Float32)
DECLARE_MULTIPLE_GETTER(Float64)
#undef DECLARE_MULTIPLE_GETTER
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const override
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeType::string)
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
@ -143,7 +143,7 @@ public:
private:
struct attribute_t final
{
AttributeType type;
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
@ -170,14 +170,13 @@ private:
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attributes.push_back(createAttributeWithType(getAttributeTypeByName(attribute.type),
attribute.null_value));
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
{
hierarchical_attribute = &attributes.back();
if (hierarchical_attribute->type != AttributeType::uint64)
if (hierarchical_attribute->type != AttributeUnderlyingType::UInt64)
throw Exception{
"Hierarchical attribute must be UInt64.",
ErrorCodes::TYPE_MISMATCH
@ -209,31 +208,31 @@ private:
}
template <typename T>
void createAttributeImpl(attribute_t & attribute, const std::string & null_value)
void createAttributeImpl(attribute_t & attribute, const Field & null_value)
{
std::get<T>(attribute.null_values) = DB::parse<T>(null_value);
std::get<T>(attribute.null_values) = null_value.get<typename NearestFieldType<T>::Type>();
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 createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
{
attribute_t attr{type};
switch (type)
{
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:
case AttributeUnderlyingType::UInt8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::String:
{
const auto & null_value_ref = std::get<String>(attr.null_values) = DB::parse<String>(null_value);
const auto & null_value_ref = std::get<String>(attr.null_values) = null_value.get<String>();
std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attr.maps) =
std::make_unique<HashMap<UInt64, StringRef>>();
attr.string_arena = std::make_unique<Arena>();
@ -268,17 +267,17 @@ private:
{
switch (attribute.type)
{
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:
case AttributeUnderlyingType::UInt8: setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt16: setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt32: setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt64: setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::Int8: setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int16: setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int32: setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int64: setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Float32: setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::Float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::String:
{
auto & map = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
const auto & string = value.get<String>();

View File

@ -4,6 +4,8 @@
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <statdaemons/ext/range.hpp>
#include <mysqlxx/Query.h>
#include <vector>
@ -15,9 +17,30 @@ namespace DB
/// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining
class MySQLBlockInputStream final : public IProfilingBlockInputStream
{
enum struct value_type_t
{
UInt8,
UInt16,
UInt32,
UInt64,
Int8,
Int16,
Int32,
Int64,
Float32,
Float64,
String,
Date,
DateTime
};
public:
MySQLBlockInputStream(mysqlxx::Query query, const Block & sample_block, const std::size_t max_block_size)
: query{std::move(query)}, result{query.use()}, sample_block{sample_block}, max_block_size{max_block_size}
MySQLBlockInputStream(const mysqlxx::PoolWithFailover::Entry & entry,
const std::string & query_str,
const Block & sample_block,
const std::size_t max_block_size)
: entry{entry}, query{this->entry->query(query_str)}, result{query.use()},
sample_block{sample_block}, max_block_size{max_block_size}
{
types.reserve(sample_block.columns());
@ -25,27 +48,31 @@ public:
{
const auto type = sample_block.getByPosition(idx).type.get();
if (typeid_cast<const DataTypeUInt8 *>(type))
types.push_back(AttributeType::uint8);
types.push_back(value_type_t::UInt8);
else if (typeid_cast<const DataTypeUInt16 *>(type))
types.push_back(AttributeType::uint16);
types.push_back(value_type_t::UInt16);
else if (typeid_cast<const DataTypeUInt32 *>(type))
types.push_back(AttributeType::uint32);
types.push_back(value_type_t::UInt32);
else if (typeid_cast<const DataTypeUInt64 *>(type))
types.push_back(AttributeType::uint64);
types.push_back(value_type_t::UInt64);
else if (typeid_cast<const DataTypeInt8 *>(type))
types.push_back(AttributeType::int8);
types.push_back(value_type_t::Int8);
else if (typeid_cast<const DataTypeInt16 *>(type))
types.push_back(AttributeType::int16);
types.push_back(value_type_t::Int16);
else if (typeid_cast<const DataTypeInt32 *>(type))
types.push_back(AttributeType::int32);
types.push_back(value_type_t::Int32);
else if (typeid_cast<const DataTypeInt64 *>(type))
types.push_back(AttributeType::int64);
types.push_back(value_type_t::Int64);
else if (typeid_cast<const DataTypeFloat32 *>(type))
types.push_back(AttributeType::float32);
types.push_back(value_type_t::Float32);
else if (typeid_cast<const DataTypeInt64 *>(type))
types.push_back(AttributeType::float64);
types.push_back(value_type_t::Float64);
else if (typeid_cast<const DataTypeString *>(type))
types.push_back(AttributeType::string);
types.push_back(value_type_t::String);
else if (typeid_cast<const DataTypeDate *>(type))
types.push_back(value_type_t::Date);
else if (typeid_cast<const DataTypeDateTime *>(type))
types.push_back(value_type_t::DateTime);
else
throw Exception{
"Unsupported type " + type->getName(),
@ -76,6 +103,7 @@ private:
std::size_t rows = 0;
while (auto row = result.fetch())
{
/// @todo cache pointers returned by the calls to getByPosition
for (const auto idx : ext::range(0, row.size()))
insertValue(block.getByPosition(idx).column, row[idx], types[idx]);
@ -87,29 +115,32 @@ private:
return rows == 0 ? Block{} : block;
};
static void insertValue(ColumnPtr & column, const mysqlxx::Value & value, const AttributeType type)
static void insertValue(ColumnPtr & column, const mysqlxx::Value & value, const value_type_t type)
{
switch (type)
{
case AttributeType::uint8: column->insert(static_cast<UInt64>(value)); break;
case AttributeType::uint16: column->insert(static_cast<UInt64>(value)); break;
case AttributeType::uint32: column->insert(static_cast<UInt64>(value)); break;
case AttributeType::uint64: column->insert(static_cast<UInt64>(value)); break;
case AttributeType::int8: column->insert(static_cast<Int64>(value)); break;
case AttributeType::int16: column->insert(static_cast<Int64>(value)); break;
case AttributeType::int32: column->insert(static_cast<Int64>(value)); break;
case AttributeType::int64: column->insert(static_cast<Int64>(value)); break;
case AttributeType::float32: column->insert(static_cast<Float64>(value)); break;
case AttributeType::float64: column->insert(static_cast<Float64>(value)); break;
case AttributeType::string: column->insert(value.getString()); break;
}
case value_type_t::UInt8: column->insert(static_cast<UInt64>(value)); break;
case value_type_t::UInt16: column->insert(static_cast<UInt64>(value)); break;
case value_type_t::UInt32: column->insert(static_cast<UInt64>(value)); break;
case value_type_t::UInt64: column->insert(static_cast<UInt64>(value)); break;
case value_type_t::Int8: column->insert(static_cast<Int64>(value)); break;
case value_type_t::Int16: column->insert(static_cast<Int64>(value)); break;
case value_type_t::Int32: column->insert(static_cast<Int64>(value)); break;
case value_type_t::Int64: column->insert(static_cast<Int64>(value)); break;
case value_type_t::Float32: column->insert(static_cast<Float64>(value)); break;
case value_type_t::Float64: column->insert(static_cast<Float64>(value)); break;
case value_type_t::String: column->insert(value.getString()); break;
case value_type_t::Date: column->insert(static_cast<UInt64>(UInt16{value.getDate().getDayNum()})); break;
case value_type_t::DateTime: column->insert(static_cast<UInt64>(time_t{value.getDateTime()})); break;
};
}
mysqlxx::PoolWithFailover::Entry entry;
mysqlxx::Query query;
mysqlxx::UseQueryResult result;
Block sample_block;
const std::size_t max_block_size;
std::vector<AttributeType> types;
std::vector<value_type_t> types;
};
}

View File

@ -36,7 +36,7 @@ public:
BlockInputStreamPtr loadAll() override
{
last_modification = getLastModification();
return new MySQLBlockInputStream{pool.Get()->query(load_all_query), sample_block, max_block_size};
return new MySQLBlockInputStream{pool.Get(), load_all_query, sample_block, max_block_size};
}
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> ids) override
@ -44,7 +44,7 @@ public:
last_modification = getLastModification();
const auto query = composeLoadIdsQuery(ids);
return new MySQLBlockInputStream{pool.Get()->query(query), sample_block, max_block_size};
return new MySQLBlockInputStream{pool.Get(), query, sample_block, max_block_size};
}
bool isModified() const override { return getLastModification() > last_modification; }
@ -56,16 +56,24 @@ private:
mysqlxx::DateTime getLastModification() const
{
const auto Update_time_idx = 12;
mysqlxx::DateTime update_time{std::time(nullptr)};
try
{
auto connection = pool.Get();
auto query = connection->query("SHOW TABLE STATUS LIKE '%" + strconvert::escaped_for_like(table) + "%';");
auto result = query.use();
auto row = result.fetch();
const auto & update_time = row[Update_time_idx];
if (!update_time.isNull())
return update_time.getDateTime();
if (auto row = result.fetch())
{
const auto & update_time_value = row[Update_time_idx];
if (!update_time_value.isNull())
update_time = update_time_value.getDateTime();
/// fetch remaining rows to avoid "commands out of sync" error
while (auto row = result.fetch());
}
}
catch (...)
{
@ -73,7 +81,7 @@ private:
}
/// we suppose failure to get modification time is not an error, therefore return current time
return mysqlxx::DateTime{std::time(nullptr)};
return update_time;
}
static std::string composeLoadAllQuery(const Block & block, const std::string & table)

View File

@ -134,6 +134,65 @@ struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
};
/** Отдельный случай для преобразования UInt32 или UInt64 в Date.
* Если число меньше 65536, то оно понимается, как DayNum, а если больше - как unix timestamp.
* Немного нелогично, что мы, по сути, помещаем две разные функции в одну.
* Но зато это позволяет поддержать распространённый случай,
* когда пользователь пишет toDate(UInt32), ожидая, что это - перевод unix timestamp в дату
* (иначе такое использование было бы распространённой ошибкой).
*/
template <typename FromDataType, typename Name>
struct ConvertImplUInt32Or64ToDate
{
typedef typename FromDataType::FieldType FromFieldType;
typedef DataTypeDate::FieldType ToFieldType;
template <typename To, typename From>
static To convert(const From & from, const DateLUT & date_lut)
{
return from < 0xFFFF
? from
: date_lut.toDayNum(from);
}
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
DateLUT & date_lut = DateLUT::instance();
if (const ColumnVector<FromFieldType> * col_from
= typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
{
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
block.getByPosition(result).column = col_to;
const typename ColumnVector<FromFieldType>::Container_t & vec_from = col_from->getData();
typename ColumnVector<ToFieldType>::Container_t & vec_to = col_to->getData();
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = convert<ToFieldType>(vec_from[i], date_lut);
}
else if (const ColumnConst<FromFieldType> * col_from
= typeid_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
{
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(),
convert<ToFieldType>(col_from->getData(), date_lut));
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of first argument of function " + Name::name,
ErrorCodes::ILLEGAL_COLUMN);
}
};
template <typename Name>
struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name> : ConvertImplUInt32Or64ToDate<DataTypeUInt32, Name> {};
template <typename Name>
struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name> : ConvertImplUInt32Or64ToDate<DataTypeUInt64, Name> {};
/** Преобразование чисел, дат, дат-с-временем в строки: через форматирование.
*/
template <typename DataType> void formatImpl(typename DataType::FieldType x, WriteBuffer & wb) { writeText(x, wb); }
@ -392,9 +451,9 @@ public:
IDataType * from_type = &*block.getByPosition(arguments[0]).type;
if (typeid_cast<const DataTypeUInt8 * >(from_type)) ConvertImpl<DataTypeUInt8, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeUInt16 * >(from_type)) ConvertImpl<DataTypeUInt16, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeUInt32 * >(from_type)) ConvertImpl<DataTypeUInt32, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeUInt64 * >(from_type)) ConvertImpl<DataTypeUInt64, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeUInt16 * >(from_type)) ConvertImpl<DataTypeUInt16, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeUInt32 * >(from_type)) ConvertImpl<DataTypeUInt32, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeUInt64 * >(from_type)) ConvertImpl<DataTypeUInt64, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeInt8 * >(from_type)) ConvertImpl<DataTypeInt8, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeInt16 * >(from_type)) ConvertImpl<DataTypeInt16, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeInt32 * >(from_type)) ConvertImpl<DataTypeInt32, ToDataType, Name>::execute(block, arguments, result);
@ -403,7 +462,7 @@ public:
else if (typeid_cast<const DataTypeFloat64 * >(from_type)) ConvertImpl<DataTypeFloat64, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeDate * >(from_type)) ConvertImpl<DataTypeDate, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeDateTime * >(from_type)) ConvertImpl<DataTypeDateTime, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeString * >(from_type)) ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeString * >(from_type)) ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
else if (typeid_cast<const DataTypeFixedString *>(from_type)) ConvertImpl<DataTypeFixedString, ToDataType, Name>::execute(block, arguments, result);
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),

View File

@ -3,6 +3,8 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnArray.h>
@ -877,6 +879,8 @@ DECLARE_DICT_GET_TRAITS(Int32, DataTypeInt32)
DECLARE_DICT_GET_TRAITS(Int64, DataTypeInt64)
DECLARE_DICT_GET_TRAITS(Float32, DataTypeFloat32)
DECLARE_DICT_GET_TRAITS(Float64, DataTypeFloat64)
DECLARE_DICT_GET_TRAITS(UInt16, DataTypeDate)
DECLARE_DICT_GET_TRAITS(UInt32, DataTypeDateTime)
#undef DECLARE_DICT_GET_TRAITS
template <typename DataType>
@ -1009,7 +1013,7 @@ private:
};
template <typename DataType>
const std::string FunctionDictGet<DataType>::name = "dictGet" + TypeName<typename DataType::FieldType>::get();
const std::string FunctionDictGet<DataType>::name = "dictGet" + DataType{}.getName();
using FunctionDictGetUInt8 = FunctionDictGet<DataTypeUInt8>;
@ -1022,6 +1026,8 @@ using FunctionDictGetInt32 = FunctionDictGet<DataTypeInt32>;
using FunctionDictGetInt64 = FunctionDictGet<DataTypeInt64>;
using FunctionDictGetFloat32 = FunctionDictGet<DataTypeFloat32>;
using FunctionDictGetFloat64 = FunctionDictGet<DataTypeFloat64>;
using FunctionDictGetDate = FunctionDictGet<DataTypeDate>;
using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime>;
class FunctionDictGetHierarchy final : public IFunction
@ -1396,4 +1402,4 @@ private:
};
}
};

View File

@ -64,6 +64,9 @@ public:
pos = ptr + offset;
}
/// получить буфер
inline Buffer & internalBuffer() { return internal_buffer; }
/// получить часть буфера, из которого можно читать / в который можно писать данные
inline Buffer & buffer() { return working_buffer; }

View File

@ -91,20 +91,26 @@ private:
if (current_memory_tracker)
current_memory_tracker->alloc(m_capacity);
char * new_m_data = nullptr;
if (!alignment)
{
m_data = reinterpret_cast<char *>(malloc(m_capacity));
new_m_data = reinterpret_cast<char *>(malloc(m_capacity));
if (!m_data)
if (!new_m_data)
throw Exception("Cannot allocate memory (malloc)", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
m_data = new_m_data;
return;
}
int res = posix_memalign(reinterpret_cast<void **>(&m_data), alignment, (m_capacity + alignment - 1) / alignment * alignment);
int res = posix_memalign(reinterpret_cast<void **>(&new_m_data), alignment, (m_capacity + alignment - 1) / alignment * alignment);
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign)", ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
m_data = new_m_data;
}
void dealloc()

View File

@ -53,12 +53,13 @@ private:
owned_cell.reset(new UncompressedCacheCell);
size_t size_decompressed;
owned_cell->compressed_size = readCompressedData(size_decompressed);
size_t size_compressed_without_checksum;
owned_cell->compressed_size = readCompressedData(size_decompressed, size_compressed_without_checksum);
if (owned_cell->compressed_size)
{
owned_cell->data.resize(size_decompressed);
decompress(owned_cell->data.m_data, size_decompressed);
decompress(owned_cell->data.m_data, size_decompressed, size_compressed_without_checksum);
/// Положим данные в кэш.
cache->set(key, owned_cell);

View File

@ -14,14 +14,15 @@ private:
bool nextImpl()
{
size_t size_decompressed;
size_compressed = readCompressedData(size_decompressed);
size_t size_compressed_without_checksum;
size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum);
if (!size_compressed)
return false;
memory.resize(size_decompressed);
working_buffer = Buffer(&memory[0], &memory[size_decompressed]);
decompress(working_buffer.begin(), size_decompressed);
decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum);
return true;
}
@ -44,14 +45,15 @@ public:
while (bytes_read < n)
{
size_t size_decompressed;
size_t size_compressed_without_checksum;
if (!readCompressedData(size_decompressed))
if (!readCompressedData(size_decompressed, size_compressed_without_checksum))
return bytes_read;
/// Если разжатый блок помещается целиком туда, куда его надо скопировать.
if (size_decompressed <= n - bytes_read)
{
decompress(to + bytes_read, size_decompressed);
decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum);
bytes_read += size_decompressed;
bytes += size_decompressed;
}
@ -62,7 +64,7 @@ public:
working_buffer = Buffer(&memory[0], &memory[size_decompressed]);
pos = working_buffer.begin();
decompress(working_buffer.begin(), size_decompressed);
decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum);
bytes_read += read(to + bytes_read, n - bytes_read);
break;

View File

@ -5,6 +5,7 @@
#include <city.h>
#include <quicklz/quicklz_level1.h>
#include <lz4/lz4.h>
#include <zstd/zstd.h>
#include <DB/Common/PODArray.h>
#include <DB/Common/ProfileEvents.h>
@ -32,7 +33,7 @@ protected:
/// Прочитать сжатые данные в compressed_buffer. Достать из их заголовка размер разжатых данных. Проверить чексумму.
/// Возвращает количество прочитанных байт.
size_t readCompressedData(size_t & size_decompressed)
size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum)
{
if (compressed_in->eof())
return 0;
@ -44,14 +45,15 @@ protected:
compressed_in->readStrict(&own_compressed_buffer[0], QUICKLZ_HEADER_SIZE);
UInt8 method = own_compressed_buffer[0]; /// См. CompressedWriteBuffer.h
size_t size_compressed;
size_t & size_compressed = size_compressed_without_checksum;
if (method < 0x80)
{
size_compressed = qlz_size_compressed(&own_compressed_buffer[0]);
size_decompressed = qlz_size_decompressed(&own_compressed_buffer[0]);
}
else if (method == 0x82)
else if (method == static_cast<UInt8>(CompressionMethodByte::LZ4) || method == static_cast<UInt8>(CompressionMethodByte::ZSTD))
{
size_compressed = *reinterpret_cast<const UInt32 *>(&own_compressed_buffer[1]);
size_decompressed = *reinterpret_cast<const UInt32 *>(&own_compressed_buffer[5]);
@ -85,7 +87,7 @@ protected:
return size_compressed + sizeof(checksum);
}
void decompress(char * to, size_t size_decompressed)
void decompress(char * to, size_t size_decompressed, size_t size_compressed_without_checksum)
{
ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks);
ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed);
@ -99,10 +101,19 @@ protected:
qlz_decompress(&compressed_buffer[0], to, qlz_state);
}
else if (method == 0x82)
else if (method == static_cast<UInt8>(CompressionMethodByte::LZ4))
{
if (LZ4_decompress_fast(&compressed_buffer[QUICKLZ_HEADER_SIZE], to, size_decompressed) < 0)
throw Exception("Cannot LZ4_decompress_fast", ErrorCodes::CORRUPTED_DATA);
throw Exception("Cannot LZ4_decompress_fast", ErrorCodes::CANNOT_DECOMPRESS);
}
else if (method == static_cast<UInt8>(CompressionMethodByte::ZSTD))
{
size_t res = ZSTD_decompress(
to, size_decompressed,
&compressed_buffer[QUICKLZ_HEADER_SIZE], size_compressed_without_checksum - QUICKLZ_HEADER_SIZE);
if (ZSTD_isError(res))
throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS);
}
else
throw Exception("Unknown compression method: " + toString(method), ErrorCodes::UNKNOWN_COMPRESSION_METHOD);

View File

@ -24,14 +24,15 @@ private:
bool nextImpl()
{
size_t size_decompressed;
size_compressed = readCompressedData(size_decompressed);
size_t size_compressed_without_checksum;
size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum);
if (!size_compressed)
return false;
memory.resize(size_decompressed);
working_buffer = Buffer(&memory[0], &memory[size_decompressed]);
decompress(working_buffer.begin(), size_decompressed);
decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum);
return true;
}
@ -81,8 +82,9 @@ public:
while (bytes_read < n)
{
size_t size_decompressed = 0;
size_t size_compressed_without_checksum = 0;
size_t new_size_compressed = readCompressedData(size_decompressed);
size_t new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum);
size_compressed = 0; /// file_in больше не указывает на конец блока в working_buffer.
if (!new_size_compressed)
return bytes_read;
@ -90,7 +92,7 @@ public:
/// Если разжатый блок помещается целиком туда, куда его надо скопировать.
if (size_decompressed <= n - bytes_read)
{
decompress(to + bytes_read, size_decompressed);
decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum);
bytes_read += size_decompressed;
bytes += size_decompressed;
}
@ -102,7 +104,7 @@ public:
working_buffer = Buffer(&memory[0], &memory[size_decompressed]);
pos = working_buffer.begin();
decompress(working_buffer.begin(), size_decompressed);
decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum);
bytes_read += read(to + bytes_read, n - bytes_read);
break;

View File

@ -11,15 +11,48 @@
namespace DB
{
namespace CompressionMethod
/** Метод сжатия */
enum class CompressionMethod
{
/** Метод сжатия */
enum Enum
{
QuickLZ,
LZ4,
LZ4HC, /// Формат такой же, как у LZ4. Разница только при сжатии.
};
}
QuickLZ,
LZ4,
LZ4HC, /// Формат такой же, как у LZ4. Разница только при сжатии.
ZSTD, /// Экспериментальный алгоритм: https://github.com/Cyan4973/zstd
};
/** Формат сжатого блока следующий:
*
* Первые 16 байт - чексумма от всех остальных байт блока. Сейчас используется только CityHash128.
* В дальнейшем можно предусмотреть другие чексуммы, хотя сделать их другого размера не получится.
*
* Следующий байт определяет алгоритм сжатия. Далее всё зависит от алгоритма.
*
* Первые 4 варианта совместимы с QuickLZ level 1.
* То есть, если значение первого байта < 4, для разжатия достаточно использовать функцию qlz_level1_decompress.
*
* 0x00 - несжатые данные, маленький блок. Далее один байт - размер сжатых данных, с учётом заголовка; один байт - размер несжатых данных.
* 0x01 - сжатые данные, QuickLZ level 1, маленький блок. Далее два байта аналогично.
* 0x02 - несжатые данные, большой блок. Далее 4 байта - размер сжатых данных, с учётом заголовка; 4 байта - размер несжатых данных.
* 0x03 - сжатые данные, QuickLZ level 1, большой блок. Далее 8 байт аналогично.
*
* 0x82 - LZ4 или LZ4HC (они имеют одинаковый формат).
* Далее 4 байта - размер сжатых данных, с учётом заголовка; 4 байта - размер несжатых данных.
*
* NOTE: Почему 0x82?
* Изначально использовался только QuickLZ. Потом был добавлен LZ4.
* Старший бит выставлен, чтобы отличить от QuickLZ, а второй бит выставлен для совместимости,
* чтобы работали функции qlz_size_compressed, qlz_size_decompressed.
* Хотя сейчас такая совместимость уже не актуальна.
*
* 0x90 - ZSTD
*
* Все размеры - little endian.
*/
enum class CompressionMethodByte : uint8_t
{
LZ4 = 0x82,
ZSTD = 0x90,
};
}

View File

@ -8,6 +8,7 @@
#include <quicklz/quicklz_level1.h>
#include <lz4/lz4.h>
#include <lz4/lz4hc.h>
#include <zstd/zstd.h>
#include <DB/Common/PODArray.h>
#include <DB/Core/Types.h>
@ -24,7 +25,7 @@ class CompressedWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
{
private:
WriteBuffer & out;
CompressionMethod::Enum method;
CompressionMethod method;
PODArray<char> compressed_buffer;
qlz_state_compress * qlz_state;
@ -38,31 +39,7 @@ private:
size_t compressed_size = 0;
char * compressed_buffer_ptr = nullptr;
/** Формат сжатого блока следующий:
*
* Первые 16 байт - чексумма от всех остальных байт блока. Сейчас используется только CityHash128.
* В дальнейшем можно предусмотреть другие чексуммы, хотя сделать их другого размера не получится.
*
* Следующий байт определяет алгоритм сжатия. Далее всё зависит от алгоритма.
*
* Первые 4 варианта совместимы с QuickLZ level 1.
* То есть, если значение первого байта < 4, для разжатия достаточно использовать функцию qlz_level1_decompress.
*
* 0x00 - несжатые данные, маленький блок. Далее один байт - размер сжатых данных, с учётом заголовка; один байт - размер несжатых данных.
* 0x01 - сжатые данные, QuickLZ level 1, маленький блок. Далее два байта аналогично.
* 0x02 - несжатые данные, большой блок. Далее 4 байта - размер сжатых данных, с учётом заголовка; 4 байта - размер несжатых данных.
* 0x03 - сжатые данные, QuickLZ level 1, большой блок. Далее 8 байт аналогично.
*
* 0x82 - LZ4 или LZ4HC (они имеют одинаковый формат).
* Далее 4 байта - размер сжатых данных, с учётом заголовка; 4 байта - размер несжатых данных.
*
* NOTE: Почему 0x82?
* Изначально использовался только QuickLZ. Потом был добавлен LZ4.
* Старший бит выставлен, чтобы отличить от QuickLZ, а второй бит выставлен для совместимости,
* чтобы работали функции qlz_size_compressed, qlz_size_decompressed.
* Хотя сейчас такая совместимость уже не актуальна.
*
* Все размеры - little endian.
/** Формат сжатого блока - см. CompressedStream.h
*/
switch (method)
@ -88,7 +65,7 @@ private:
compressed_buffer.resize(header_size + LZ4_COMPRESSBOUND(uncompressed_size));
compressed_buffer[0] = 0x82; /// Второй бит - для совместимости с QuickLZ - обозначает, что размеры записываются 4 байтами.
compressed_buffer[0] = static_cast<UInt8>(CompressionMethodByte::LZ4);
if (method == CompressionMethod::LZ4)
compressed_size = header_size + LZ4_compress(
@ -110,6 +87,34 @@ private:
compressed_buffer_ptr = &compressed_buffer[0];
break;
}
case CompressionMethod::ZSTD:
{
static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32);
compressed_buffer.resize(header_size + ZSTD_compressBound(uncompressed_size));
compressed_buffer[0] = static_cast<UInt8>(CompressionMethodByte::ZSTD);
size_t res = ZSTD_compress(
&compressed_buffer[header_size],
compressed_buffer.size(),
working_buffer.begin(),
uncompressed_size);
if (ZSTD_isError(res))
throw Exception("Cannot compress block with ZSTD: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_COMPRESS);
compressed_size = header_size + res;
UInt32 compressed_size_32 = compressed_size;
UInt32 uncompressed_size_32 = uncompressed_size;
memcpy(&compressed_buffer[1], reinterpret_cast<const char *>(&compressed_size_32), sizeof(compressed_size_32));
memcpy(&compressed_buffer[5], reinterpret_cast<const char *>(&uncompressed_size_32), sizeof(uncompressed_size_32));
compressed_buffer_ptr = &compressed_buffer[0];
break;
}
default:
throw Exception("Unknown compression method", ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
}
@ -123,7 +128,7 @@ private:
public:
CompressedWriteBuffer(
WriteBuffer & out_,
CompressionMethod::Enum method_ = CompressionMethod::LZ4,
CompressionMethod method_ = CompressionMethod::LZ4,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE)
: BufferWithOwnMemory<WriteBuffer>(buf_size), out(out_), method(method_), qlz_state(new qlz_state_compress) {}

View File

@ -0,0 +1,69 @@
#pragma once
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
#include <statdaemons/AIO.h>
#include <string>
#include <limits>
#include <unistd.h>
#include <fcntl.h>
namespace DB
{
/** Класс для асинхронной чтения данных.
* Все размеры и смещения должны быть кратны DEFAULT_AIO_FILE_BLOCK_SIZE байтам.
*/
class ReadBufferAIO : public BufferWithOwnMemory<ReadBuffer>
{
public:
ReadBufferAIO(const std::string & filename_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags_ = -1, mode_t mode_ = 0666,
char * existing_memory_ = nullptr);
~ReadBufferAIO() override;
ReadBufferAIO(const ReadBufferAIO &) = delete;
ReadBufferAIO & operator=(const ReadBufferAIO &) = delete;
void setMaxBytes(size_t max_bytes_read_);
off_t seek(off_t off, int whence = SEEK_SET);
off_t getPositionInFile();
std::string getFileName() const noexcept { return filename; }
int getFD() const noexcept { return fd; }
private:
off_t getPositionInFileRelaxed() const noexcept;
bool nextImpl();
/// Ждать окончания текущей асинхронной задачи.
void waitForAIOCompletion();
/// Менять местами основной и дублирующий буферы.
void swapBuffers() noexcept;
private:
/// Буфер для асинхронных операций чтения данных.
BufferWithOwnMemory<ReadBuffer> fill_buffer;
iocb request;
std::vector<iocb *> request_ptrs;
std::vector<io_event> events;
AIOContext aio_context;
const std::string filename;
size_t max_bytes_read = std::numeric_limits<size_t>::max();
size_t total_bytes_read = 0;
off_t pos_in_file = 0;
int fd = -1;
/// Асинхронная операция чтения ещё не завершилась.
bool is_pending_read = false;
/// Было получено исключение.
bool got_exception = false;
/// Конец файла достигнут.
bool is_eof = false;
/// Был отправлен хоть один запрос на асинхронную операцию чтения.
bool is_started = false;
};
}

View File

@ -99,7 +99,7 @@ public:
}
}
size_t getPositionInFile()
off_t getPositionInFile()
{
return pos_in_file - (working_buffer.end() - pos);
}

View File

@ -0,0 +1,65 @@
#pragma once
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
#include <statdaemons/AIO.h>
#include <string>
#include <unistd.h>
#include <fcntl.h>
namespace DB
{
/** Класс для асинхронной записи данных.
* Все размеры и смещения должны быть кратны DEFAULT_AIO_FILE_BLOCK_SIZE байтам.
*/
class WriteBufferAIO : public BufferWithOwnMemory<WriteBuffer>
{
public:
WriteBufferAIO(const std::string & filename_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags_ = -1, mode_t mode_ = 0666,
char * existing_memory_ = nullptr);
~WriteBufferAIO() override;
WriteBufferAIO(const WriteBufferAIO &) = delete;
WriteBufferAIO & operator=(const WriteBufferAIO &) = delete;
off_t seek(off_t off, int whence = SEEK_SET);
off_t getPositionInFile();
void truncate(off_t length = 0);
void sync();
std::string getFileName() const noexcept { return filename; }
int getFD() const noexcept { return fd; }
private:
/// Если в буфере ещё остались данные - запишем их.
void flush();
///
void nextImpl();
/// Ждать окончания текущей асинхронной задачи.
void waitForAIOCompletion();
/// Менять местами основной и дублирующий буферы.
void swapBuffers() noexcept;
private:
/// Буфер для асинхронных операций записи данных.
BufferWithOwnMemory<WriteBuffer> flush_buffer;
iocb request;
std::vector<iocb *> request_ptrs;
std::vector<io_event> events;
AIOContext aio_context;
const std::string filename;
off_t pos_in_file = 0;
int fd = -1;
/// Асинхронная операция записи ещё не завершилась.
bool is_pending_write = false;
/// Было получено исключение.
bool got_exception = false;
};
}

View File

@ -528,9 +528,7 @@ struct AggregatedDataVariants : private boost::noncopyable
void init(Type type_)
{
type = type_;
switch (type)
switch (type_)
{
case Type::EMPTY: break;
case Type::without_key: break;
@ -543,6 +541,8 @@ struct AggregatedDataVariants : private boost::noncopyable
default:
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
type = type_;
}
size_t size() const
@ -676,19 +676,6 @@ APPLY_FOR_AGGREGATED_VARIANTS(M)
class Aggregator
{
public:
Aggregator(const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_,
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, Compiler * compiler_, UInt32 min_count_to_compile_,
size_t group_by_two_level_threshold_)
: keys(keys_), aggregates(aggregates_), aggregates_size(aggregates.size()),
overflow_row(overflow_row_),
max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_),
compiler(compiler_), min_count_to_compile(min_count_to_compile_), group_by_two_level_threshold(group_by_two_level_threshold_)
{
std::sort(keys.begin(), keys.end());
keys.erase(std::unique(keys.begin(), keys.end()), keys.end());
keys_size = keys.size();
}
Aggregator(const Names & key_names_, const AggregateDescriptions & aggregates_, bool overflow_row_,
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, Compiler * compiler_, UInt32 min_count_to_compile_,
size_t group_by_two_level_threshold_)
@ -739,6 +726,9 @@ public:
/// Для IBlockInputStream.
String getID() const;
size_t getNumberOfKeys() const { return keys_size; }
size_t getNumberOfAggregates() const { return aggregates_size; }
protected:
friend struct AggregatedDataVariants;
@ -945,7 +935,7 @@ protected:
template <typename Method, typename Table>
void mergeStreamsImpl(
Block & block,
AggregatedDataVariants & result,
const Sizes & key_sizes,
Arena * aggregates_pool,
Method & method,
Table & data) const;

View File

@ -218,6 +218,9 @@ private:
/// Eliminates injective function calls and constant expressions from group by statement
void optimizeGroupBy();
/// Удалить из ORDER BY повторяющиеся элементы.
void optimizeOrderBy();
/// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn.
void makeSet(ASTFunction * node, const Block & sample_block);

View File

@ -39,7 +39,10 @@ private:
static const auto check_period_sec = 5;
mutable std::mutex dictionaries_mutex;
std::unordered_map<std::string, std::shared_ptr<MultiVersion<IDictionary>>> dictionaries;
using dictionary_ptr_t = std::shared_ptr<MultiVersion<IDictionary>>;
using dictionary_origin_pair_t = std::pair<dictionary_ptr_t, std::string>;
std::unordered_map<std::string, dictionary_origin_pair_t> dictionaries;
/// exception pointers for notifying user about failures on dictionary creation
std::unordered_map<std::string, std::exception_ptr> stored_exceptions;
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
@ -52,9 +55,10 @@ private:
Logger * log;
Poco::Timestamp config_last_modified{0};
std::unordered_map<std::string, Poco::Timestamp> last_modification_times;
void reloadImpl();
void reloadFromFile(const std::string & config_path);
void reloadPeriodically()
{
@ -105,7 +109,7 @@ public:
};
}
return it->second->get();
return it->second.first->get();
}
};

View File

@ -109,6 +109,12 @@ struct Settings
\
/** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \
\
/** Минимальное количество байтов для операций ввода/ввывода минуя кэш страниц */ \
M(SettingUInt64, min_bytes_to_use_direct_io, (20U * 1024U * 1024U * 1024U)) \
\
/** Кидать исключение, если есть индекс по дате, и он не используется. */ \
M(SettingBool, force_index_by_date, 0) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

@ -235,11 +235,11 @@ void NO_INLINE Aggregator::executeSpecializedCase(
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
if (inserted)
{
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = nullptr;
method.onNewKey(*it, keys_size, i, keys, *aggregates_pool);
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = nullptr;
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
AggregateFunctionsList::forEach(AggregateFunctionsCreator(

View File

@ -47,29 +47,19 @@ public:
/// Основное имя типа таблицы (например, StorageMergeTree).
virtual std::string getName() const = 0;
/** Возвращает true, если хранилище получает данные с удалённого сервера или серверов.
*/
/** Возвращает true, если хранилище получает данные с удалённого сервера или серверов. */
virtual bool isRemote() const { return false; }
virtual void storeExternalTables(const std::map<String, StoragePtr> & tables_)
{
throw Exception("Method storeExternalTables is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Возвращает true, если хранилище поддерживает запросы с секцией SAMPLE.
*/
/** Возвращает true, если хранилище поддерживает запросы с секцией SAMPLE. */
virtual bool supportsSampling() const { return false; }
/** Возвращает true, если хранилище поддерживает запросы с секцией FINAL.
*/
/** Возвращает true, если хранилище поддерживает запросы с секцией FINAL. */
virtual bool supportsFinal() const { return false; }
/** Возвращает true, если хранилище поддерживает запросы с секцией PREWHERE.
*/
/** Возвращает true, если хранилище поддерживает запросы с секцией PREWHERE. */
virtual bool supportsPrewhere() const { return false; }
/** Возвращает true, если хранилище поддерживает несколько реплик.
*/
/** Возвращает true, если хранилище поддерживает несколько реплик. */
virtual bool supportsParallelReplicas() const { return false; }
/** Не дает изменять описание таблицы (в том числе переименовывать и удалять таблицу).

View File

@ -55,6 +55,8 @@ namespace DB
* (см. CollapsingSortedBlockInputStream.h)
* - Summing - при склейке кусков, при совпадении PK суммировать все числовые столбцы, не входящие в PK.
* - Aggregating - при склейке кусков, при совпадении PK, делается слияние состояний столбцов-агрегатных функций.
* - Unsorted - при склейке кусков, данные не упорядочиваются, а всего лишь конкатенируются;
* - это позволяет читать данные ровно такими пачками, какими они были записаны.
*/
/** Этот класс хранит список кусков и параметры структуры данных.
@ -399,18 +401,22 @@ public:
}
size_t key_size = storage.sort_descr.size();
index.resize(key_size * size);
String index_path = storage.full_path + name + "/primary.idx";
ReadBufferFromFile index_file(index_path,
std::min(static_cast<size_t>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(index_path).getSize()));
if (key_size)
{
index.resize(key_size * size);
for (size_t i = 0; i < size; ++i)
for (size_t j = 0; j < key_size; ++j)
storage.primary_key_sample.getByPosition(j).type->deserializeBinary(index[i * key_size + j], index_file);
String index_path = storage.full_path + name + "/primary.idx";
ReadBufferFromFile index_file(index_path,
std::min(static_cast<size_t>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(index_path).getSize()));
if (!index_file.eof())
throw Exception("index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE);
for (size_t i = 0; i < size; ++i)
for (size_t j = 0; j < key_size; ++j)
storage.primary_key_sample.getByPosition(j).type->deserializeBinary(index[i * key_size + j], index_file);
if (!index_file.eof())
throw Exception("index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE);
}
size_in_bytes = calcTotalSize(storage.full_path + name + "/");
}
@ -468,7 +474,7 @@ public:
if (!checksums.empty())
{
if (!checksums.files.count("primary.idx"))
if (!storage.sort_descr.empty() && !checksums.files.count("primary.idx"))
throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART);
if (require_part_metadata)
@ -486,12 +492,14 @@ public:
}
else
{
/// Проверяем, что первичный ключ непуст.
if (!storage.sort_descr.empty())
{
/// Проверяем, что первичный ключ непуст.
Poco::File index_file(path + "/primary.idx");
Poco::File index_file(path + "/primary.idx");
if (!index_file.exists() || index_file.getSize() == 0)
throw Exception("Part " + path + " is broken: primary key is empty.", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
if (!index_file.exists() || index_file.getSize() == 0)
throw Exception("Part " + path + " is broken: primary key is empty.", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
}
/// Проверяем, что все засечки непусты и имеют одинаковый размер.
@ -620,6 +628,7 @@ public:
Collapsing,
Summing,
Aggregating,
Unsorted,
};
static void doNothing(const String & name) {}
@ -628,7 +637,7 @@ public:
* (корректность имён и путей не проверяется)
* состоящую из указанных столбцов.
*
* primary_expr_ast - выражение для сортировки;
* primary_expr_ast - выражение для сортировки; Пустое для UnsortedMergeTree.
* date_column_name - имя столбца с датой;
* index_granularity - на сколько строчек пишется одно значение индекса.
* require_part_metadata - обязательно ли в директории с куском должны быть checksums.txt и columns.txt

View File

@ -128,8 +128,11 @@ public:
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ALL_REQUESTED_COLUMNS_ARE_MISSING)
if (e.code() != ErrorCodes::ALL_REQUESTED_COLUMNS_ARE_MISSING
&& e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
{
storage.reportBrokenPart(part_name);
}
/// Более хорошая диагностика.
throw Exception(e.message() + "\n(while reading from part " + path + " from mark " + toString(from_mark) + " to "
@ -258,8 +261,10 @@ public:
/// remove added column to ensure same content among all blocks
if (added_column)
{
res.erase(0);
streams.erase(added_column->name);
columns.erase(std::begin(columns));
added_column = nullptr;
}
/// sort columns to ensure consistent order among all blocks
@ -280,12 +285,6 @@ public:
std::swap(res, ordered_block);
}
else if (added_column)
{
res.erase(0);
}
added_column = nullptr;
}
catch (const Exception & e)
{

View File

@ -15,18 +15,32 @@ namespace DB
class IMergedBlockOutputStream : public IBlockOutputStream
{
public:
IMergedBlockOutputStream(MergeTreeData & storage_, size_t min_compress_block_size_, size_t max_compress_block_size_) : storage(storage_), index_offset(0), min_compress_block_size(min_compress_block_size_), max_compress_block_size(max_compress_block_size_)
IMergedBlockOutputStream(
MergeTreeData & storage_,
size_t min_compress_block_size_,
size_t max_compress_block_size_,
CompressionMethod compression_method_)
: storage(storage_),
min_compress_block_size(min_compress_block_size_),
max_compress_block_size(max_compress_block_size_),
compression_method(compression_method_)
{
}
protected:
typedef std::set<std::string> OffsetColumns;
using OffsetColumns = std::set<std::string>;
struct ColumnStream
{
ColumnStream(const String & escaped_column_name_, const String & data_path, const std::string & marks_path, size_t max_compress_block_size = DEFAULT_MAX_COMPRESS_BLOCK_SIZE) :
ColumnStream(
const String & escaped_column_name_,
const String & data_path,
const std::string & marks_path,
size_t max_compress_block_size,
CompressionMethod compression_method) :
escaped_column_name(escaped_column_name_),
plain_file(data_path, max_compress_block_size, O_TRUNC | O_CREAT | O_WRONLY),
plain_hashing(plain_file), compressed_buf(plain_hashing), compressed(compressed_buf),
plain_hashing(plain_file), compressed_buf(plain_hashing, compression_method), compressed(compressed_buf),
marks_file(marks_path, 4096, O_TRUNC | O_CREAT | O_WRONLY), marks(marks_file) {}
String escaped_column_name;
@ -70,7 +84,7 @@ protected:
}
};
typedef std::map<String, std::unique_ptr<ColumnStream> > ColumnStreams;
using ColumnStreams = std::map<String, std::unique_ptr<ColumnStream>>;
void addStream(const String & path, const String & name, const IDataType & type, size_t level = 0, String filename = "")
{
@ -92,7 +106,8 @@ protected:
escaped_size_name,
path + escaped_size_name + ".bin",
path + escaped_size_name + ".mrk",
max_compress_block_size));
max_compress_block_size,
compression_method));
addStream(path, name, *type_arr->getNestedType(), level + 1);
}
@ -101,7 +116,8 @@ protected:
escaped_column_name,
path + escaped_column_name + ".bin",
path + escaped_column_name + ".mrk",
max_compress_block_size));
max_compress_block_size,
compression_method));
}
@ -146,7 +162,8 @@ protected:
type_arr->serializeOffsets(column, stream.compressed, prev_mark, limit);
stream.compressed.nextIfAtEnd(); /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего.
/// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего.
stream.compressed.nextIfAtEnd();
prev_mark += limit;
}
@ -180,7 +197,8 @@ protected:
type.serializeBinary(column, stream.compressed, prev_mark, limit);
stream.compressed.nextIfAtEnd(); /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего.
/// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего.
stream.compressed.nextIfAtEnd();
prev_mark += limit;
}
@ -192,10 +210,12 @@ protected:
ColumnStreams column_streams;
/// Смещение до первой строчки блока, для которой надо записать индекс.
size_t index_offset;
size_t index_offset = 0;
size_t min_compress_block_size;
size_t max_compress_block_size;
CompressionMethod compression_method;
};
/** Для записи одного куска. Данные уже отсортированы, относятся к одному месяцу, и пишутся в один кускок.
@ -203,13 +223,23 @@ protected:
class MergedBlockOutputStream : public IMergedBlockOutputStream
{
public:
MergedBlockOutputStream(MergeTreeData & storage_, String part_path_, const NamesAndTypesList & columns_list_)
: IMergedBlockOutputStream(storage_, storage_.context.getSettings().min_compress_block_size, storage_.context.getSettings().max_compress_block_size), columns_list(columns_list_), part_path(part_path_), marks_count(0)
MergedBlockOutputStream(
MergeTreeData & storage_,
String part_path_,
const NamesAndTypesList & columns_list_,
CompressionMethod compression_method)
: IMergedBlockOutputStream(
storage_, storage_.context.getSettings().min_compress_block_size,
storage_.context.getSettings().max_compress_block_size, compression_method),
columns_list(columns_list_), part_path(part_path_)
{
Poco::File(part_path).createDirectories();
index_file_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY);
index_stream = new HashingWriteBuffer(*index_file_stream);
if (storage.mode != MergeTreeData::Unsorted)
{
index_file_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY);
index_stream = new HashingWriteBuffer(*index_file_stream);
}
for (const auto & it : columns_list)
addStream(part_path, it.name, *it.type);
@ -233,7 +263,9 @@ public:
{
for (PrimaryColumns::const_iterator it = primary_columns.begin(); it != primary_columns.end(); ++it)
{
index_vec.push_back((*(*it)->column)[i]);
if (storage.mode != MergeTreeData::Unsorted)
index_vec.push_back((*(*it)->column)[i]);
(*it)->type->serializeBinary(index_vec.back(), *index_stream);
}
@ -264,9 +296,13 @@ public:
/// Заканчиваем запись и достаем чексуммы.
MergeTreeData::DataPart::Checksums checksums;
index_stream->next();
checksums.files["primary.idx"].file_size = index_stream->count();
checksums.files["primary.idx"].file_hash = index_stream->getHash();
if (storage.mode != MergeTreeData::Unsorted)
{
index_stream->next();
checksums.files["primary.idx"].file_size = index_stream->count();
checksums.files["primary.idx"].file_hash = index_stream->getHash();
index_stream = nullptr;
}
for (ColumnStreams::iterator it = column_streams.begin(); it != column_streams.end(); ++it)
{
@ -274,7 +310,6 @@ public:
it->second->addToChecksums(checksums);
}
index_stream = nullptr;
column_streams.clear();
if (marks_count == 0)
@ -315,7 +350,7 @@ private:
NamesAndTypesList columns_list;
String part_path;
size_t marks_count;
size_t marks_count = 0;
SharedPtr<WriteBufferFromFile> index_file_stream;
SharedPtr<HashingWriteBuffer> index_stream;
@ -328,8 +363,11 @@ typedef Poco::SharedPtr<MergedBlockOutputStream> MergedBlockOutputStreamPtr;
class MergedColumnOnlyOutputStream : public IMergedBlockOutputStream
{
public:
MergedColumnOnlyOutputStream(MergeTreeData & storage_, String part_path_, bool sync_ = false) :
IMergedBlockOutputStream(storage_, storage_.context.getSettings().min_compress_block_size, storage_.context.getSettings().max_compress_block_size), part_path(part_path_), initialized(false), sync(sync_)
MergedColumnOnlyOutputStream(MergeTreeData & storage_, String part_path_, bool sync_, CompressionMethod compression_method)
: IMergedBlockOutputStream(
storage_, storage_.context.getSettings().min_compress_block_size,
storage_.context.getSettings().max_compress_block_size, compression_method),
part_path(part_path_), sync(sync_)
{
}
@ -386,8 +424,7 @@ public:
private:
String part_path;
bool initialized;
bool initialized = false;
bool sync;
};

View File

@ -53,25 +53,26 @@ public:
~StorageChunkMerger() override;
private:
String this_database;
String name;
const String this_database;
const String name;
NamesAndTypesListPtr columns;
String source_database;
const String source_database;
OptimizedRegularExpression table_name_regexp;
std::string destination_name_prefix;
size_t chunks_to_merge;
const size_t chunks_to_merge;
Context & context;
Settings settings;
std::thread merge_thread;
Poco::Event cancel_merge_thread;
Logger * log;
volatile bool shutdown_called;
/// Название виртуального столбца, отвечающего за имя таблицы, из которой идет чтение. (Например "_table")
String _table_column_name;
class MergeTask;
using MergeTaskPtr = std::shared_ptr<MergeTask>;
MergeTaskPtr merge_task;
DB::BackgroundProcessingPool::TaskHandle merge_task_handle;
StorageChunkMerger(
const std::string & this_database_,
const std::string & name_,
@ -85,11 +86,6 @@ private:
size_t chunks_to_merge_,
Context & context_);
void mergeThread();
bool maybeMergeSomething();
Storages selectChunksToMerge();
bool mergeChunks(const Storages & chunks);
Block getBlockWithVirtualColumns(const Storages & selected_tables) const;
typedef std::set<std::string> TableNames;

View File

@ -55,8 +55,6 @@ public:
bool hasColumn(const String & column_name) const override;
bool isRemote() const override { return true; }
/// Сохранить временные таблицы, чтобы при следующем вызове метода read переслать их на удаленные серверы.
void storeExternalTables(const Tables & tables_) override { external_tables = tables_; }
BlockInputStreams read(
const Names & column_names,
@ -121,10 +119,6 @@ private:
Context & context;
/// Временные таблицы, которые необходимо отправить на сервер. Переменная очищается после каждого вызова метода read
/// Для подготовки к отправке нужно использовтаь метод storeExternalTables
Tables external_tables;
/// Используется только, если таблица должна владеть объектом Cluster, которым больше никто не владеет - для реализации TableFunctionRemote.
SharedPtr<Cluster> owned_cluster;

View File

@ -13,7 +13,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <zkutil/ZooKeeper.h>
#include <zkutil/LeaderElection.h>
#include <statdaemons/threadpool.hpp>
namespace DB
{

View File

@ -14,6 +14,7 @@
#include <DB/AggregateFunctions/AggregateFunctionArray.h>
#include <DB/AggregateFunctions/AggregateFunctionState.h>
#include <DB/AggregateFunctions/AggregateFunctionMerge.h>
#include <DB/AggregateFunctions/AggregateFunctionDebug.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
@ -230,7 +231,9 @@ static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name
AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const
{
if (name == "count")
if (name == "debug")
return new AggregateFunctionDebug;
else if (name == "count")
return new AggregateFunctionCount;
else if (name == "any")
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types);
@ -602,6 +605,7 @@ bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int
{
static const char * names[]
{
"debug",
"count",
"any",
"anyLast",

View File

@ -56,9 +56,9 @@ public:
Benchmark(unsigned concurrency_, double delay_,
const String & host_, UInt16 port_, const String & default_database_,
const String & user_, const String & password_, const Settings & settings_)
: concurrency(concurrency_), delay(delay_), queue(concurrency), pool(concurrency),
: concurrency(concurrency_), delay(delay_), queue(concurrency),
connections(concurrency, host_, port_, default_database_, user_, password_, data_type_factory),
settings(settings_)
settings(settings_), pool(concurrency)
{
std::cerr << std::fixed << std::setprecision(3);
@ -78,8 +78,6 @@ private:
typedef ConcurrentBoundedQueue<Query> Queue;
Queue queue;
boost::threadpool::pool pool;
DataTypeFactory data_type_factory;
ConnectionPool connections;
Settings settings;
@ -123,6 +121,8 @@ private:
Poco::FastMutex mutex;
boost::threadpool::pool pool;
void readQueries()
{

View File

@ -162,11 +162,37 @@ void Connection::forceConnected()
}
}
struct PingTimeoutSetter
{
PingTimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & ping_timeout_)
: socket(socket_), ping_timeout(ping_timeout_)
{
old_send_timeout = socket.getSendTimeout();
old_receive_timeout = socket.getReceiveTimeout();
if (old_send_timeout > ping_timeout)
socket.setSendTimeout(ping_timeout);
if (old_receive_timeout > ping_timeout)
socket.setReceiveTimeout(ping_timeout);
}
~PingTimeoutSetter()
{
socket.setSendTimeout(old_send_timeout);
socket.setReceiveTimeout(old_receive_timeout);
}
Poco::Net::StreamSocket & socket;
Poco::Timespan ping_timeout;
Poco::Timespan old_send_timeout;
Poco::Timespan old_receive_timeout;
};
bool Connection::ping()
{
// LOG_TRACE(log_wrapper.get(), "Ping (" << getServerAddress() << ")");
PingTimeoutSetter timeout_setter(socket, ping_timeout);
try
{
UInt64 pong = 0;

View File

@ -43,6 +43,14 @@ ExceptionPtr cloneCurrentException()
}
}
inline std::string demangle(const char * const mangled, int & status)
{
const auto demangled_str = abi::__cxa_demangle(mangled, 0, 0, &status);
std::string demangled{demangled_str};
free(demangled_str);
return demangled;
}
void tryLogCurrentException(const char * log_name)
{
@ -72,10 +80,8 @@ void tryLogCurrentException(const char * log_name)
{
try
{
int status;
char * realname = abi::__cxa_demangle(typeid(e).name(), 0, 0, &status);
std::string name = realname;
free(realname);
int status = 0;
auto name = demangle(typeid(e).name(), status);
if (status)
name += " (demangling status: " + toString(status) + ")";
@ -88,7 +94,13 @@ void tryLogCurrentException(const char * log_name)
{
try
{
LOG_ERROR(&Logger::get(log_name), "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION);
int status = 0;
auto name = demangle(abi::__cxa_current_exception_type()->name(), status);
if (status)
name += " (demangling status: " + toString(status) + ")";
LOG_ERROR(&Logger::get(log_name), "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name);
}
catch (...) {}
}

View File

@ -12,14 +12,14 @@ CollapsingFinalBlockInputStream::~CollapsingFinalBlockInputStream()
/// Нужно обезвредить все MergingBlockPtr, чтобы они не пытались класть блоки в output_blocks.
previous.block.cancel();
last_positive.block.cancel();
while (!queue.empty())
{
Cursor c = queue.top();
queue.pop();
c.block.cancel();
}
for (size_t i = 0; i < output_blocks.size(); ++i)
delete output_blocks[i];
}
@ -43,7 +43,7 @@ void CollapsingFinalBlockInputStream::fetchNextBlock(size_t input_index)
Block block = stream->read();
if (!block)
return;
MergingBlockPtr merging_block(new MergingBlock(block, input_index, description, sign_column, &output_blocks));
MergingBlockPtr merging_block(new MergingBlock(block, input_index, description, sign_column_name, &output_blocks));
++blocks_fetched;
queue.push(Cursor(merging_block));
}
@ -56,18 +56,18 @@ void CollapsingFinalBlockInputStream::commitCurrent()
{
last_positive.addToFilter();
}
if (!(count_positive == count_negative || count_positive + 1 == count_negative || count_positive == count_negative + 1))
{
if (count_incorrect_data < MAX_ERROR_MESSAGES)
reportBadCounts();
++count_incorrect_data;
}
last_positive = Cursor();
previous = Cursor();
}
count_negative = 0;
count_positive = 0;
}
@ -81,7 +81,7 @@ Block CollapsingFinalBlockInputStream::readImpl()
first = false;
}
/// Будем формировать блоки для ответа, пока не получится непустой блок.
while (true)
{
@ -89,10 +89,10 @@ Block CollapsingFinalBlockInputStream::readImpl()
{
Cursor current = queue.top();
queue.pop();
bool has_next = !queue.empty();
Cursor next = has_next ? queue.top() : Cursor();
/// Будем продвигаться в текущем блоке, не используя очередь, пока возможно.
while (true)
{
@ -101,7 +101,7 @@ Block CollapsingFinalBlockInputStream::readImpl()
commitCurrent();
previous = current;
}
Int8 sign = current.getSign();
if (sign == 1)
{
@ -116,53 +116,50 @@ Block CollapsingFinalBlockInputStream::readImpl()
}
else
reportBadSign(sign);
if (current.isLast())
{
fetchNextBlock(current.block->stream_index);
/// Все потоки кончились. Обработаем последний ключ.
if (!has_next)
{
commitCurrent();
}
break;
}
else
{
current.next();
if (has_next && !(next < current))
{
queue.push(current);
break;
}
}
}
}
/// Конец потока.
if (output_blocks.empty())
{
if (blocks_fetched != blocks_output)
LOG_ERROR(log, "Logical error: CollapsingFinalBlockInputStream has output " << blocks_output << " blocks instead of " << blocks_fetched);
return Block();
}
MergingBlock * merging_block = output_blocks.back();
Block block = merging_block->block;
for (size_t i = 0; i < block.columns(); ++i)
block.getByPosition(i).column = block.getByPosition(i).column->filter(merging_block->filter);
output_blocks.pop_back();
delete merging_block;
++blocks_output;
if (block)
return block;
}

View File

@ -31,7 +31,7 @@ Block IProfilingBlockInputStream::read()
Block res;
if (is_cancelled)
if (is_cancelled.load(std::memory_order_seq_cst))
return res;
res = readImpl();
@ -226,7 +226,7 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
size_t rows_processed = process_list_elem->progress.rows;
size_t bytes_processed = process_list_elem->progress.bytes;
size_t total_rows_estimate = std::max(process_list_elem->progress.rows, process_list_elem->progress.total_rows);
size_t total_rows_estimate = std::max(rows_processed, process_list_elem->progress.total_rows);
/** Проверяем ограничения на объём данных для чтения, скорость выполнения запроса, квоту на объём данных для чтения.
* NOTE: Может быть, имеет смысл сделать, чтобы они проверялись прямо в ProcessList?
@ -253,16 +253,32 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
if (limits.min_execution_speed)
size_t total_rows = process_list_elem->progress.total_rows;
if (limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0))
{
double total_elapsed = info.total_stopwatch.elapsedSeconds();
if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0
&& rows_processed / total_elapsed < limits.min_execution_speed)
if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0)
{
throw Exception("Query is executing too slow: " + toString(rows_processed / total_elapsed)
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
ErrorCodes::TOO_SLOW);
if (limits.min_execution_speed && rows_processed / total_elapsed < limits.min_execution_speed)
throw Exception("Query is executing too slow: " + toString(rows_processed / total_elapsed)
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
ErrorCodes::TOO_SLOW);
size_t total_rows = process_list_elem->progress.total_rows;
/// Если предсказанное время выполнения больше, чем max_execution_time.
if (limits.max_execution_time != 0 && total_rows)
{
double estimated_execution_time_seconds = total_elapsed * (static_cast<double>(total_rows) / rows_processed);
if (estimated_execution_time_seconds > limits.max_execution_time.totalSeconds())
throw Exception("Estimated query execution time (" + toString(estimated_execution_time_seconds) + " seconds)"
+ " is too long. Maximum: " + toString(limits.max_execution_time.totalSeconds())
+ ". Estimated rows to process: " + toString(total_rows),
ErrorCodes::TOO_SLOW);
}
}
}
@ -276,7 +292,8 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
void IProfilingBlockInputStream::cancel()
{
if (!__sync_bool_compare_and_swap(&is_cancelled, false, true))
bool old_val = false;
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
for (auto & child : children)

View File

@ -7,20 +7,30 @@
namespace DB
{
void copyData(IBlockInputStream & from, IBlockOutputStream & to, volatile bool * is_cancelled)
namespace
{
bool isAtomicSet(std::atomic<bool> * val)
{
return ((val != nullptr) && val->load(std::memory_order_seq_cst));
}
}
void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic<bool> * is_cancelled)
{
from.readPrefix();
to.writePrefix();
while (Block block = from.read())
{
if (is_cancelled && *is_cancelled)
if (isAtomicSet(is_cancelled))
break;
to.write(block);
}
if (is_cancelled && *is_cancelled)
if (isAtomicSet(is_cancelled))
return;
/// Для вывода дополнительной информации в некоторых форматах.
@ -33,7 +43,7 @@ void copyData(IBlockInputStream & from, IBlockOutputStream & to, volatile bool *
to.setExtremes(input->getExtremes());
}
if (is_cancelled && *is_cancelled)
if (isAtomicSet(is_cancelled))
return;
from.readSuffix();

View File

@ -23,16 +23,18 @@
int main(int argc, char ** argv)
{
using namespace DB;
try
{
size_t n = argc == 2 ? atoi(argv[1]) : 10;
DB::Block block;
Block block;
DB::ColumnWithNameAndType column_x;
ColumnWithNameAndType column_x;
column_x.name = "x";
column_x.type = new DB::DataTypeInt16;
DB::ColumnInt16 * x = new DB::ColumnInt16;
column_x.type = new DataTypeInt16;
ColumnInt16 * x = new ColumnInt16;
column_x.column = x;
auto & vec_x = x->getData();
@ -44,65 +46,64 @@ int main(int argc, char ** argv)
const char * strings[] = {"abc", "def", "abcd", "defg", "ac"};
DB::ColumnWithNameAndType column_s1;
ColumnWithNameAndType column_s1;
column_s1.name = "s1";
column_s1.type = new DB::DataTypeString;
column_s1.column = new DB::ColumnString;
column_s1.type = new DataTypeString;
column_s1.column = new ColumnString;
for (size_t i = 0; i < n; ++i)
column_s1.column->insert(std::string(strings[i % 5]));
block.insert(column_s1);
DB::ColumnWithNameAndType column_s2;
ColumnWithNameAndType column_s2;
column_s2.name = "s2";
column_s2.type = new DB::DataTypeString;
column_s2.column = new DB::ColumnString;
column_s2.type = new DataTypeString;
column_s2.column = new ColumnString;
for (size_t i = 0; i < n; ++i)
column_s2.column->insert(std::string(strings[i % 3]));
block.insert(column_s2);
DB::ColumnNumbers key_column_numbers;
key_column_numbers.push_back(0);
//key_column_numbers.push_back(1);
Names key_column_names;
key_column_names.emplace_back("x");
DB::AggregateFunctionFactory factory;
AggregateFunctionFactory factory;
DB::AggregateDescriptions aggregate_descriptions(1);
AggregateDescriptions aggregate_descriptions(1);
DB::DataTypes empty_list_of_types;
DataTypes empty_list_of_types;
aggregate_descriptions[0].function = factory.get("count", empty_list_of_types);
Poco::SharedPtr<DB::DataTypes> result_types = new DB::DataTypes
Poco::SharedPtr<DataTypes> result_types = new DataTypes
{
new DB::DataTypeInt16,
// new DB::DataTypeString,
new DB::DataTypeUInt64,
new DataTypeInt16,
// new DataTypeString,
new DataTypeUInt64,
};
DB::Block sample;
for (DB::DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it)
Block sample;
for (DataTypes::const_iterator it = result_types->begin(); it != result_types->end(); ++it)
{
DB::ColumnWithNameAndType col;
ColumnWithNameAndType col;
col.type = *it;
sample.insert(col);
}
DB::BlockInputStreamPtr stream = new DB::OneBlockInputStream(block);
stream = new DB::AggregatingBlockInputStream(stream, key_column_numbers, aggregate_descriptions, false, true,
0, DB::OverflowMode::THROW, nullptr, 0, 0);
BlockInputStreamPtr stream = new OneBlockInputStream(block);
stream = new AggregatingBlockInputStream(stream, key_column_names, aggregate_descriptions, false, true,
0, OverflowMode::THROW, nullptr, 0, 0);
DB::WriteBufferFromOStream ob(std::cout);
DB::RowOutputStreamPtr row_out = new DB::TabSeparatedRowOutputStream(ob, sample);
DB::BlockOutputStreamPtr out = new DB::BlockOutputStreamFromRowOutputStream(row_out);
WriteBufferFromOStream ob(std::cout);
RowOutputStreamPtr row_out = new TabSeparatedRowOutputStream(ob, sample);
BlockOutputStreamPtr out = new BlockOutputStreamFromRowOutputStream(row_out);
{
Poco::Stopwatch stopwatch;
stopwatch.start();
DB::copyData(*stream, *out);
copyData(*stream, *out);
stopwatch.stop();
std::cout << std::fixed << std::setprecision(2)
@ -115,7 +116,7 @@ int main(int argc, char ** argv)
stream->dumpTree(std::cout);
std::cout << std::endl;
}
catch (const DB::Exception & e)
catch (const Exception & e)
{
std::cerr << e.displayText() << std::endl;
}

View File

@ -31,6 +31,8 @@ void registerFunctionsDictionaries(FunctionFactory & factory)
factory.registerFunction<FunctionDictGetInt64>();
factory.registerFunction<FunctionDictGetFloat32>();
factory.registerFunction<FunctionDictGetFloat64>();
factory.registerFunction<FunctionDictGetDate>();
factory.registerFunction<FunctionDictGetDateTime>();
factory.registerFunction<FunctionDictGetString>();
factory.registerFunction<FunctionDictGetHierarchy>();
factory.registerFunction<FunctionDictIsIn>();

View File

@ -0,0 +1,222 @@
#include <DB/IO/ReadBufferAIO.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/Defines.h>
#include <sys/types.h>
#include <sys/stat.h>
namespace DB
{
ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, mode_t mode_,
char * existing_memory_)
: BufferWithOwnMemory(buffer_size_, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE),
fill_buffer(BufferWithOwnMemory(buffer_size_, nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)),
request_ptrs{ &request }, events(1), filename(filename_)
{
ProfileEvents::increment(ProfileEvents::FileOpen);
int open_flags = (flags_ == -1) ? O_RDONLY : flags_;
open_flags |= O_DIRECT;
fd = ::open(filename.c_str(), open_flags, mode_);
if (fd == -1)
{
got_exception = true;
auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE;
throwFromErrno("Cannot open file " + filename, error_code);
}
::memset(&request, 0, sizeof(request));
}
ReadBufferAIO::~ReadBufferAIO()
{
if (!got_exception)
{
try
{
waitForAIOCompletion();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (fd != -1)
::close(fd);
}
void ReadBufferAIO::setMaxBytes(size_t max_bytes_read_)
{
if (is_started)
{
got_exception = true;
throw Exception("Illegal attempt to set the maximum number of bytes to read from file " + filename, ErrorCodes::LOGICAL_ERROR);
}
if ((max_bytes_read_ % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0)
{
got_exception = true;
throw Exception("Invalid maximum number of bytes to read from file " + filename, ErrorCodes::AIO_UNALIGNED_SIZE_ERROR);
}
max_bytes_read = max_bytes_read_;
}
off_t ReadBufferAIO::seek(off_t off, int whence)
{
if ((off % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0)
throw Exception("Invalid offset for ReadBufferAIO::seek", ErrorCodes::AIO_UNALIGNED_SIZE_ERROR);
waitForAIOCompletion();
off_t new_pos;
if (whence == SEEK_SET)
{
if (off < 0)
{
got_exception = true;
throw Exception("SEEK_SET underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
new_pos = off;
}
else if (whence == SEEK_CUR)
{
if (off >= 0)
{
if (off > (std::numeric_limits<off_t>::max() - getPositionInFileRelaxed()))
{
got_exception = true;
throw Exception("SEEK_CUR overflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
}
else if (off < -getPositionInFileRelaxed())
{
got_exception = true;
throw Exception("SEEK_CUR underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
new_pos = getPositionInFileRelaxed() + off;
}
else
{
got_exception = true;
throw Exception("ReadBufferAIO::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
if (new_pos != getPositionInFileRelaxed())
{
off_t working_buffer_begin_pos = pos_in_file - static_cast<off_t>(working_buffer.size());
if (hasPendingData() && (new_pos >= working_buffer_begin_pos) && (new_pos <= pos_in_file))
{
/// Свдинулись, но остались в пределах буфера.
pos = working_buffer.begin() + (new_pos - working_buffer_begin_pos);
}
else
{
pos = working_buffer.end();
pos_in_file = new_pos;
}
}
return new_pos;
}
off_t ReadBufferAIO::getPositionInFile()
{
return seek(0, SEEK_CUR);
}
off_t ReadBufferAIO::getPositionInFileRelaxed() const noexcept
{
return pos_in_file - (working_buffer.end() - pos);
}
bool ReadBufferAIO::nextImpl()
{
/// Если конец файла уже был достигнут при вызове этой функции,
/// то текущий вызов ошибочен.
if (is_eof)
return false;
waitForAIOCompletion();
/// При первом вызове не надо обменять местами основной и дублирующий буферы.
if (is_started)
swapBuffers();
else
is_started = true;
/// Если конец файла только что достигнут, больше ничего не делаем.
if (is_eof)
return true;
/// Создать запрос.
request.aio_lio_opcode = IOCB_CMD_PREAD;
request.aio_fildes = fd;
request.aio_buf = reinterpret_cast<UInt64>(fill_buffer.internalBuffer().begin());
request.aio_nbytes = std::min(fill_buffer.internalBuffer().size(), max_bytes_read);
request.aio_offset = pos_in_file;
request.aio_reqprio = 0;
/// Отправить запрос.
while (io_submit(aio_context.ctx, request_ptrs.size(), &request_ptrs[0]) < 0)
if (errno != EINTR)
{
got_exception = true;
throw Exception("Cannot submit request for asynchronous IO on file " + filename, ErrorCodes::AIO_SUBMIT_ERROR);
}
is_pending_read = true;
return true;
}
void ReadBufferAIO::waitForAIOCompletion()
{
if (is_pending_read)
{
while (io_getevents(aio_context.ctx, events.size(), events.size(), &events[0], nullptr) < 0)
if (errno != EINTR)
{
got_exception = true;
throw Exception("Failed to wait for asynchronous IO completion on file " + filename, ErrorCodes::AIO_COMPLETION_ERROR);
}
is_pending_read = false;
off_t bytes_read = events[0].res;
if (bytes_read < 0)
{
got_exception = true;
throw Exception("Asynchronous read error on file " + filename, ErrorCodes::AIO_READ_ERROR);
}
if ((bytes_read % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0)
{
got_exception = true;
throw Exception("Received unaligned number of bytes from file " + filename, ErrorCodes::AIO_UNALIGNED_SIZE_ERROR);
}
if (pos_in_file > (std::numeric_limits<off_t>::max() - bytes_read))
{
got_exception = true;
throw Exception("File position overflowed", ErrorCodes::LOGICAL_ERROR);
}
pos_in_file += bytes_read;
total_bytes_read += bytes_read;
if (bytes_read > 0)
fill_buffer.buffer().resize(bytes_read);
if ((static_cast<size_t>(bytes_read) < fill_buffer.internalBuffer().size()) || (total_bytes_read == max_bytes_read))
is_eof = true;
}
}
void ReadBufferAIO::swapBuffers() noexcept
{
internalBuffer().swap(fill_buffer.internalBuffer());
buffer().swap(fill_buffer.buffer());
std::swap(position(), fill_buffer.position());
}
}

View File

@ -0,0 +1,202 @@
#include <DB/IO/WriteBufferAIO.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Core/Defines.h>
#include <limits>
#include <sys/types.h>
#include <sys/stat.h>
namespace DB
{
WriteBufferAIO::WriteBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, mode_t mode_,
char * existing_memory_)
: BufferWithOwnMemory(buffer_size_, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE),
flush_buffer(BufferWithOwnMemory(buffer_size_, nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)),
request_ptrs{ &request }, events(1), filename(filename_)
{
ProfileEvents::increment(ProfileEvents::FileOpen);
int open_flags = (flags_ == -1) ? (O_WRONLY | O_TRUNC | O_CREAT) : flags_;
open_flags |= O_DIRECT;
fd = ::open(filename.c_str(), open_flags, mode_);
if (fd == -1)
{
got_exception = true;
auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE;
throwFromErrno("Cannot open file " + filename, error_code);
}
::memset(&request, 0, sizeof(request));
}
WriteBufferAIO::~WriteBufferAIO()
{
if (!got_exception)
{
try
{
flush();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (fd != -1)
::close(fd);
}
off_t WriteBufferAIO::seek(off_t off, int whence)
{
if ((off % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0)
throw Exception("Invalid offset for WriteBufferAIO::seek", ErrorCodes::AIO_UNALIGNED_SIZE_ERROR);
flush();
if (whence == SEEK_SET)
{
if (off < 0)
{
got_exception = true;
throw Exception("SEEK_SET underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
pos_in_file = off;
}
else if (whence == SEEK_CUR)
{
if (off >= 0)
{
if (off > (std::numeric_limits<off_t>::max() - pos_in_file))
{
got_exception = true;
throw Exception("SEEK_CUR overflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
}
else if (off < -pos_in_file)
{
got_exception = true;
throw Exception("SEEK_CUR underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
pos_in_file += off;
}
else
{
got_exception = true;
throw Exception("WriteBufferAIO::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
return pos_in_file;
}
off_t WriteBufferAIO::getPositionInFile()
{
return seek(0, SEEK_CUR);
}
void WriteBufferAIO::truncate(off_t length)
{
if ((length % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0)
throw Exception("Invalid length for WriteBufferAIO::ftruncate", ErrorCodes::AIO_UNALIGNED_SIZE_ERROR);
flush();
int res = ::ftruncate(fd, length);
if (res == -1)
{
got_exception = true;
throwFromErrno("Cannot truncate file " + filename, ErrorCodes::CANNOT_TRUNCATE_FILE);
}
}
void WriteBufferAIO::sync()
{
flush();
/// Попросим ОС сбросить данные на диск.
int res = ::fsync(fd);
if (res == -1)
{
got_exception = true;
throwFromErrno("Cannot fsync " + getFileName(), ErrorCodes::CANNOT_FSYNC);
}
}
void WriteBufferAIO::flush()
{
next();
waitForAIOCompletion();
}
void WriteBufferAIO::nextImpl()
{
if (!offset())
return;
waitForAIOCompletion();
swapBuffers();
/// Создать запрос.
request.aio_lio_opcode = IOCB_CMD_PWRITE;
request.aio_fildes = fd;
request.aio_buf = reinterpret_cast<UInt64>(flush_buffer.buffer().begin());
request.aio_nbytes = flush_buffer.offset();
request.aio_offset = pos_in_file;
request.aio_reqprio = 0;
if ((request.aio_nbytes % DEFAULT_AIO_FILE_BLOCK_SIZE) != 0)
{
got_exception = true;
throw Exception("Illegal attempt to write unaligned data to file " + filename, ErrorCodes::AIO_UNALIGNED_SIZE_ERROR);
}
/// Отправить запрос.
while (io_submit(aio_context.ctx, request_ptrs.size(), &request_ptrs[0]) < 0)
if (errno != EINTR)
{
got_exception = true;
throw Exception("Cannot submit request for asynchronous IO on file " + filename, ErrorCodes::AIO_SUBMIT_ERROR);
}
is_pending_write = true;
}
void WriteBufferAIO::waitForAIOCompletion()
{
if (is_pending_write)
{
while (io_getevents(aio_context.ctx, events.size(), events.size(), &events[0], nullptr) < 0)
if (errno != EINTR)
{
got_exception = true;
throw Exception("Failed to wait for asynchronous IO completion on file " + filename, ErrorCodes::AIO_COMPLETION_ERROR);
}
is_pending_write = false;
off_t bytes_written = events[0].res;
if ((bytes_written < 0) || (static_cast<size_t>(bytes_written) < flush_buffer.offset()))
{
got_exception = true;
throw Exception("Asynchronous write error on file " + filename, ErrorCodes::AIO_WRITE_ERROR);
}
if (pos_in_file > (std::numeric_limits<off_t>::max() - bytes_written))
{
got_exception = true;
throw Exception("File position overflowed", ErrorCodes::LOGICAL_ERROR);
}
pos_in_file += bytes_written;
}
}
void WriteBufferAIO::swapBuffers() noexcept
{
buffer().swap(flush_buffer.buffer());
std::swap(position(), flush_buffer.position());
}
}

View File

@ -0,0 +1,337 @@
#include <DB/IO/ReadBufferAIO.h>
#include <DB/Core/Defines.h>
#include <boost/filesystem.hpp>
#include <vector>
#include <iostream>
#include <fstream>
#include <functional>
#include <cstdlib>
#include <unistd.h>
namespace
{
void run();
void prepare(std::string & directory, std::string & filename, std::string & buf);
void die(const std::string & msg);
void run_test(unsigned int num, const std::function<bool()> func);
bool test1(const std::string & filename);
bool test2(const std::string & filename, const std::string & buf);
bool test3(const std::string & filename, const std::string & buf);
bool test4(const std::string & filename, const std::string & buf);
bool test5(const std::string & filename);
bool test6(const std::string & filename, const std::string & buf);
bool test7(const std::string & filename, const std::string & buf);
bool test8(const std::string & filename);
bool test9(const std::string & filename, const std::string & buf);
bool test10(const std::string & filename, const std::string & buf);
bool test11(const std::string & filename);
bool test12(const std::string & filename, const std::string & buf);
void run()
{
namespace fs = boost::filesystem;
std::string directory;
std::string filename;
std::string buf;
prepare(directory, filename, buf);
const std::vector<std::function<bool()> > tests =
{
std::bind(test1, std::ref(filename)),
std::bind(test2, std::ref(filename), std::ref(buf)),
std::bind(test3, std::ref(filename), std::ref(buf)),
std::bind(test4, std::ref(filename), std::ref(buf)),
std::bind(test5, std::ref(filename)),
std::bind(test6, std::ref(filename), std::ref(buf)),
std::bind(test7, std::ref(filename), std::ref(buf)),
std::bind(test8, std::ref(filename)),
std::bind(test9, std::ref(filename), std::ref(buf)),
std::bind(test10, std::ref(filename), std::ref(buf)),
std::bind(test11, std::ref(filename)),
std::bind(test12, std::ref(filename), std::ref(buf))
};
unsigned int num = 0;
for (const auto & test : tests)
{
++num;
run_test(num, test);
}
fs::remove_all(directory);
}
void prepare(std::string & directory, std::string & filename, std::string & buf)
{
static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
if (dir == nullptr)
die("Could not create directory");
directory = std::string(dir);
filename = directory + "/foo";
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
buf.reserve(n);
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
std::ofstream out(filename.c_str());
if (!out.is_open())
die("Could not open file");
out << buf;
}
void die(const std::string & msg)
{
std::cout << msg << "\n";
::exit(EXIT_FAILURE);
}
void run_test(unsigned int num, const std::function<bool()> func)
{
bool ok;
try
{
ok = func();
}
catch (const DB::Exception & ex)
{
ok = false;
std::cout << "Caught exception " << ex.displayText() << "\n";
}
catch (const std::exception & ex)
{
ok = false;
std::cout << "Caught exception " << ex.what() << "\n";
}
if (ok)
std::cout << "Test " << num << " passed\n";
else
std::cout << "Test " << num << " failed\n";
}
bool test1(const std::string & filename)
{
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
if (in.getFileName() != filename)
return false;
if (in.getFD() == -1)
return false;
return true;
}
bool test2(const std::string & filename, const std::string & buf)
{
std::string newbuf;
newbuf.resize(buf.length());
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
size_t count = in.read(&newbuf[0], newbuf.length());
if (count != newbuf.length())
return false;
return (newbuf == buf);
}
bool test3(const std::string & filename, const std::string & buf)
{
std::string newbuf;
newbuf.resize(buf.length());
size_t requested = 9 * DEFAULT_AIO_FILE_BLOCK_SIZE;
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
in.setMaxBytes(requested);
size_t count = in.read(&newbuf[0], newbuf.length());
newbuf.resize(count);
return (newbuf == buf.substr(0, requested));
}
bool test4(const std::string & filename, const std::string & buf)
{
std::string newbuf;
newbuf.resize(buf.length());
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
in.setMaxBytes(0);
size_t n_read = in.read(&newbuf[0], newbuf.length());
return n_read == 0;
}
bool test5(const std::string & filename)
{
bool ok = false;
try
{
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
in.setMaxBytes(DEFAULT_AIO_FILE_BLOCK_SIZE >> 1);
}
catch (const DB::Exception &)
{
ok = true;
}
return ok;
}
bool test6(const std::string & filename, const std::string & buf)
{
std::string newbuf;
newbuf.resize(buf.length());
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
if (in.getPositionInFile() != 0)
return false;
size_t count = in.read(&newbuf[0], newbuf.length());
if (count != newbuf.length())
return false;
if (static_cast<size_t>(in.getPositionInFile()) != buf.length())
return false;
return true;
}
bool test7(const std::string & filename, const std::string & buf)
{
std::string newbuf;
newbuf.resize(buf.length() - DEFAULT_AIO_FILE_BLOCK_SIZE);
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
(void) in.seek(DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_SET);
size_t count = in.read(&newbuf[0], newbuf.length());
if (count != (9 * DEFAULT_AIO_FILE_BLOCK_SIZE))
return false;
return (newbuf == buf.substr(DEFAULT_AIO_FILE_BLOCK_SIZE));
}
bool test8(const std::string & filename)
{
bool ok = false;
try
{
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
(void) in.seek(DEFAULT_AIO_FILE_BLOCK_SIZE + 1, SEEK_CUR);
}
catch (const DB::Exception &)
{
ok = true;
}
return ok;
}
bool test9(const std::string & filename, const std::string & buf)
{
bool ok = false;
try
{
std::string newbuf;
newbuf.resize(buf.length());
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
size_t count = in.read(&newbuf[0], newbuf.length());
if (count != newbuf.length())
return false;
in.setMaxBytes(9 * DEFAULT_AIO_FILE_BLOCK_SIZE);
}
catch (const DB::Exception &)
{
ok = true;
}
return ok;
}
bool test10(const std::string & filename, const std::string & buf)
{
std::string newbuf;
newbuf.resize(4 * DEFAULT_AIO_FILE_BLOCK_SIZE);
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
size_t count1 = in.read(&newbuf[0], newbuf.length());
if (count1 != newbuf.length())
return false;
if (newbuf != buf.substr(0, 4 * DEFAULT_AIO_FILE_BLOCK_SIZE))
return false;
(void) in.seek(2 * DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_CUR);
size_t count2 = in.read(&newbuf[0], newbuf.length());
if (count2 != newbuf.length())
return false;
if (newbuf != buf.substr(6 * DEFAULT_AIO_FILE_BLOCK_SIZE))
return false;
return true;
}
bool test11(const std::string & filename)
{
bool ok = false;
try
{
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
(void) in.seek(-DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_SET);
}
catch (const DB::Exception &)
{
ok = true;
}
return ok;
}
bool test12(const std::string & filename, const std::string & buf)
{
bool ok = false;
try
{
std::string newbuf;
newbuf.resize(4 * DEFAULT_AIO_FILE_BLOCK_SIZE);
DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
size_t count = in.read(&newbuf[0], newbuf.length());
if (count != newbuf.length())
return false;
(void) in.seek(-(10 * DEFAULT_AIO_FILE_BLOCK_SIZE), SEEK_CUR);
}
catch (const DB::Exception &)
{
ok = true;
}
return ok;
}
}
int main()
{
run();
return 0;
}

View File

@ -0,0 +1,291 @@
#include <DB/IO/WriteBufferAIO.h>
#include <DB/Core/Defines.h>
#include <boost/filesystem.hpp>
#include <iostream>
#include <fstream>
#include <streambuf>
#include <cstdlib>
namespace
{
void run();
void die(const std::string & msg);
void run_test(unsigned int num, const std::function<bool()> func);
bool test1();
bool test2();
bool test3();
bool test4();
void run()
{
const std::vector<std::function<bool()> > tests =
{
test1,
test2,
test3,
test4
};
unsigned int num = 0;
for (const auto & test : tests)
{
++num;
run_test(num, test);
}
}
void die(const std::string & msg)
{
std::cout << msg;
::exit(EXIT_FAILURE);
}
void run_test(unsigned int num, const std::function<bool()> func)
{
bool ok;
try
{
ok = func();
}
catch (const DB::Exception & ex)
{
ok = false;
std::cout << "Caught exception " << ex.displayText() << "\n";
}
catch (const std::exception & ex)
{
ok = false;
std::cout << "Caught exception " << ex.what() << "\n";
}
if (ok)
std::cout << "Test " << num << " passed\n";
else
std::cout << "Test " << num << " failed\n";
}
bool test1()
{
namespace fs = boost::filesystem;
static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
if (dir == nullptr)
die("Could not create directory");
const std::string directory = std::string(dir);
const std::string filename = directory + "/foo";
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
std::string buf;
buf.reserve(n);
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
if (out.getFileName() != filename)
return false;
if (out.getFD() == -1)
return false;
out.write(&buf[0], buf.length());
}
std::ifstream in(filename.c_str());
if (!in.is_open())
die("Could not open file");
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
return (received == buf);
}
bool test2()
{
namespace fs = boost::filesystem;
static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
if (dir == nullptr)
die("Could not create directory");
const std::string directory = std::string(dir);
const std::string filename = directory + "/foo";
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
std::string buf;
buf.reserve(n);
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
if (out.getFileName() != filename)
return false;
if (out.getFD() == -1)
return false;
out.write(&buf[0], buf.length() / 2);
out.seek(DEFAULT_AIO_FILE_BLOCK_SIZE, SEEK_CUR);
out.write(&buf[buf.length() / 2], buf.length() / 2);
}
std::ifstream in(filename.c_str());
if (!in.is_open())
die("Could not open file");
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
if (received.substr(0, buf.length() / 2) != buf.substr(0, buf.length() / 2))
return false;
if (received.substr(buf.length() / 2, DEFAULT_AIO_FILE_BLOCK_SIZE) != std::string(DEFAULT_AIO_FILE_BLOCK_SIZE, '\0'))
return false;
if (received.substr(buf.length() / 2 + DEFAULT_AIO_FILE_BLOCK_SIZE) != buf.substr(buf.length() / 2))
return false;
return true;
}
bool test3()
{
namespace fs = boost::filesystem;
static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
if (dir == nullptr)
die("Could not create directory");
const std::string directory = std::string(dir);
const std::string filename = directory + "/foo";
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
std::string buf;
buf.reserve(n);
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
if (out.getFileName() != filename)
return false;
if (out.getFD() == -1)
return false;
out.write(&buf[0], buf.length());
off_t pos1 = out.getPositionInFile();
out.truncate(buf.length() / 2);
off_t pos2 = out.getPositionInFile();
if (pos1 != pos2)
return false;
}
std::ifstream in(filename.c_str());
if (!in.is_open())
die("Could not open file");
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
return (received == buf.substr(0, buf.length() / 2));
}
bool test4()
{
namespace fs = boost::filesystem;
static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
if (dir == nullptr)
die("Could not create directory");
const std::string directory = std::string(dir);
const std::string filename = directory + "/foo";
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
std::string buf;
buf.reserve(n);
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
if (out.getFileName() != filename)
return false;
if (out.getFD() == -1)
return false;
out.write(&buf[0], buf.length());
off_t pos1 = out.getPositionInFile();
out.truncate(3 * buf.length() / 2);
off_t pos2 = out.getPositionInFile();
if (pos1 != pos2)
return false;
}
std::ifstream in(filename.c_str());
if (!in.is_open())
die("Could not open file");
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
if (received.substr(0, buf.length()) != buf)
return false;
if (received.substr(buf.length()) != std::string(buf.length() / 2, '\0'))
return false;
return true;
}
}
int main()
{
run();
return 0;
}

View File

@ -482,12 +482,13 @@ void NO_INLINE Aggregator::executeImplCase(
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
if (inserted)
{
method.onNewKey(*it, keys_size, i, keys, *aggregates_pool);
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
/// exception-safety - если не удалось выделить память или создать состояния, то не будут вызываться деструкторы.
aggregate_data = nullptr;
method.onNewKey(*it, keys_size, i, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);
aggregate_data = place;
@ -981,20 +982,30 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
/// packaged_task используются, чтобы исключения автоматически прокидывались в основной поток.
std::vector<std::packaged_task<Block()>> tasks;
tasks.reserve(Method::Data::NUM_BUCKETS);
std::vector<std::packaged_task<Block()>> tasks(Method::Data::NUM_BUCKETS);
for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
try
{
if (method.data.impls[bucket].empty())
continue;
for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
{
if (method.data.impls[bucket].empty())
continue;
tasks.emplace_back(std::bind(converter, bucket, current_memory_tracker));
tasks[bucket] = std::packaged_task<Block()>(std::bind(converter, bucket, current_memory_tracker));
if (thread_pool)
thread_pool->schedule([bucket, &tasks] { tasks[bucket](); });
else
tasks[bucket]();
}
}
catch (...)
{
/// Если этого не делать, то в случае исключения, tasks уничтожится раньше завершения потоков, и будет плохо.
if (thread_pool)
thread_pool->schedule([bucket, &tasks] { tasks[bucket](); });
else
tasks[bucket]();
thread_pool->wait();
throw;
}
if (thread_pool)
@ -1008,6 +1019,9 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
std::exception_ptr first_exception;
for (auto & task : tasks)
{
if (!task.valid())
continue;
try
{
blocks.emplace_back(task.get_future().get());
@ -1054,15 +1068,35 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
&& data_variants.isTwoLevel()) /// TODO Использовать общий тред-пул с функцией merge.
thread_pool.reset(new boost::threadpool::pool(max_threads));
if (data_variants.type == AggregatedDataVariants::Type::without_key || overflow_row)
blocks.splice(blocks.end(), prepareBlocksAndFillWithoutKey(data_variants, final));
if (data_variants.type != AggregatedDataVariants::Type::without_key)
try
{
if (!data_variants.isTwoLevel())
blocks.splice(blocks.end(), prepareBlocksAndFillSingleLevel(data_variants, final));
else
blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get()));
if (data_variants.type == AggregatedDataVariants::Type::without_key || overflow_row)
blocks.splice(blocks.end(), prepareBlocksAndFillWithoutKey(data_variants, final));
if (data_variants.type != AggregatedDataVariants::Type::without_key)
{
if (!data_variants.isTwoLevel())
blocks.splice(blocks.end(), prepareBlocksAndFillSingleLevel(data_variants, final));
else
blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get()));
}
}
catch (...)
{
/** Если был хотя бы один эксепшен, то следует "откатить" владение состояниями агрегатных функций в ColumnAggregateFunction-ах
* - то есть, очистить их (см. комментарий в функции prepareBlockAndFill.)
*/
for (auto & block : blocks)
{
for (size_t column_num = keys_size; column_num < keys_size + aggregates_size; ++column_num)
{
IColumn & col = *block.getByPosition(column_num).column;
if (ColumnAggregateFunction * col_aggregate = typeid_cast<ColumnAggregateFunction *>(&col))
col_aggregate->getData().clear();
}
}
throw;
}
if (!final)
@ -1112,13 +1146,13 @@ void NO_INLINE Aggregator::mergeDataImpl(
for (size_t i = 0; i < aggregates_size; ++i)
aggregate_functions[i]->destroy(
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
Method::getAggregateData(it->second) = nullptr;
}
else
{
res_it->second = it->second;
}
Method::getAggregateData(it->second) = nullptr;
}
}
@ -1194,24 +1228,35 @@ void NO_INLINE Aggregator::mergeTwoLevelDataImpl(
/// packaged_task используются, чтобы исключения автоматически прокидывались в основной поток.
std::vector<std::packaged_task<void()>> tasks;
tasks.reserve(Method::Data::NUM_BUCKETS);
std::vector<std::packaged_task<void()>> tasks(Method::Data::NUM_BUCKETS);
for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
try
{
tasks.emplace_back(std::bind(merge_bucket, bucket, current_memory_tracker));
for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
{
tasks[bucket] = std::packaged_task<void()>(std::bind(merge_bucket, bucket, current_memory_tracker));
if (thread_pool)
thread_pool->schedule([bucket, &tasks] { tasks[bucket](); });
else
tasks[bucket]();
}
}
catch (...)
{
/// Если этого не делать, то в случае исключения, tasks уничтожится раньше завершения потоков, и будет плохо.
if (thread_pool)
thread_pool->schedule([bucket, &tasks] { tasks[bucket](); });
else
tasks[bucket]();
thread_pool->wait();
throw;
}
if (thread_pool)
thread_pool->wait();
for (auto & task : tasks)
task.get_future().get();
if (task.valid())
task.get_future().get();
}
@ -1333,7 +1378,7 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va
template <typename Method, typename Table>
void NO_INLINE Aggregator::mergeStreamsImpl(
Block & block,
AggregatedDataVariants & result,
const Sizes & key_sizes,
Arena * aggregates_pool,
Method & method,
Table & data) const
@ -1360,17 +1405,17 @@ void NO_INLINE Aggregator::mergeStreamsImpl(
bool inserted; /// Вставили новый ключ, или такой ключ уже был?
/// Получаем ключ для вставки в хэш-таблицу.
auto key = state.getKey(key_columns, keys_size, i, result.key_sizes, keys, *aggregates_pool);
auto key = state.getKey(key_columns, keys_size, i, key_sizes, keys, *aggregates_pool);
data.emplace(key, it, inserted);
if (inserted)
{
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = nullptr;
method.onNewKey(*it, keys_size, i, keys, *aggregates_pool);
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = nullptr;
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);
aggregate_data = place;
@ -1426,9 +1471,6 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
Block empty_block;
initialize(empty_block);
/// result будет уничтожать состояния агрегатных функций в деструкторе
result.aggregator = this;
/** Если на удалённых серверах использовался двухуровневый метод агрегации,
* то в блоках будет расположена информация о номере корзины.
* Тогда вычисления можно будет распараллелить по корзинам.
@ -1468,7 +1510,8 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
* - в случае одноуровневой агрегации, а также для блоков с "переполнившимися" значениями.
* Если есть хотя бы один блок с номером корзины больше нуля, значит была двухуровневая агрегация.
*/
size_t has_two_level = bucket_to_blocks.rbegin()->first > 0;
UInt32 max_bucket = bucket_to_blocks.rbegin()->first;
size_t has_two_level = max_bucket > 0;
if (has_two_level)
{
@ -1481,6 +1524,9 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
#undef M
}
/// result будет уничтожать состояния агрегатных функций в деструкторе
result.aggregator = this;
result.init(method);
result.keys_size = keys_size;
result.key_sizes = key_sizes;
@ -1492,12 +1538,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
{
LOG_TRACE(log, "Merging partially aggregated two-level data.");
std::unique_ptr<boost::threadpool::pool> thread_pool;
if (max_threads > 1 && total_input_rows > 100000 /// TODO Сделать настраиваемый порог.
&& has_two_level)
thread_pool.reset(new boost::threadpool::pool(max_threads));
auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker)
auto merge_bucket = [&bucket_to_blocks, &result, &key_sizes, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker)
{
current_memory_tracker = memory_tracker;
@ -1505,7 +1546,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
{
#define M(NAME) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(block, result, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket]);
mergeStreamsImpl(block, key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket]);
if (false) {}
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
@ -1517,12 +1558,16 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
/// packaged_task используются, чтобы исключения автоматически прокидывались в основной поток.
std::vector<std::packaged_task<void()>> tasks;
tasks.reserve(bucket_to_blocks.size() - has_blocks_with_unknown_bucket);
std::vector<std::packaged_task<void()>> tasks(max_bucket + 1);
for (auto & bucket_blocks : bucket_to_blocks)
std::unique_ptr<boost::threadpool::pool> thread_pool;
if (max_threads > 1 && total_input_rows > 100000 /// TODO Сделать настраиваемый порог.
&& has_two_level)
thread_pool.reset(new boost::threadpool::pool(max_threads));
for (const auto & bucket_blocks : bucket_to_blocks)
{
auto bucket = bucket_blocks.first;
const auto bucket = bucket_blocks.first;
if (bucket == -1)
continue;
@ -1530,7 +1575,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
result.aggregates_pools.push_back(new Arena);
Arena * aggregates_pool = result.aggregates_pools.back().get();
tasks.emplace_back(std::bind(merge_bucket, bucket, aggregates_pool, current_memory_tracker));
tasks[bucket] = std::packaged_task<void()>(std::bind(merge_bucket, bucket, aggregates_pool, current_memory_tracker));
if (thread_pool)
thread_pool->schedule([bucket, &tasks] { tasks[bucket](); });
@ -1542,7 +1587,8 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
thread_pool->wait();
for (auto & task : tasks)
task.get_future().get();
if (task.valid())
task.get_future().get();
LOG_TRACE(log, "Merged partially aggregated two-level data.");
}
@ -1559,7 +1605,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(block, result, result.aggregates_pool, *result.NAME, result.NAME->data);
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data);
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M

View File

@ -41,10 +41,10 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
}
else if ("cache" == layout_type)
{
const auto size = config.getInt(layout_prefix + ".cache.size");
const auto size = config.getInt(layout_prefix + ".cache.size_in_cells");
if (size == 0)
throw Exception{
"Dictionary of type 'cache' cannot have size of 0 bytes",
"Dictionary of type 'cache' cannot have 0 cells",
ErrorCodes::TOO_SMALL_BUFFER_SIZE
};

View File

@ -85,6 +85,9 @@ void ExpressionAnalyzer::init()
/// GROUP BY injective function elimination.
optimizeGroupBy();
/// Удалить из ORDER BY повторяющиеся элементы.
optimizeOrderBy();
/// array_join_alias_to_name, array_join_result_to_source.
getArrayJoinedColumns();
@ -162,11 +165,13 @@ void ExpressionAnalyzer::analyzeAggregation()
}
NameAndTypePair key{column_name, col.type};
aggregation_keys.push_back(key);
/// Ключи агрегации уникализируются.
if (!unique_keys.count(key.name))
{
unique_keys.insert(key.name);
aggregation_keys.push_back(key);
/// key is no longer needed, therefore we can save a little by moving it
aggregated_columns.push_back(std::move(key));
}
@ -529,6 +534,38 @@ void ExpressionAnalyzer::optimizeGroupBy()
}
void ExpressionAnalyzer::optimizeOrderBy()
{
if (!(select_query && select_query->order_expression_list))
return;
/// Уникализируем условия сортировки.
using NameAndLocale = std::pair<std::string, std::string>;
std::set<NameAndLocale> elems_set;
ASTs & elems = select_query->order_expression_list->children;
ASTs unique_elems;
unique_elems.reserve(elems.size());
for (const auto & elem : elems)
{
String name = elem->children.front()->getColumnName();
const ASTOrderByElement & order_by_elem = typeid_cast<const ASTOrderByElement &>(*elem);
if (elems_set.emplace(
std::piecewise_construct,
std::forward_as_tuple(name),
std::forward_as_tuple(order_by_elem.collator ? order_by_elem.collator->getLocale() : std::string())).second)
{
unique_elems.emplace_back(elem);
}
}
if (unique_elems.size() < elems.size())
elems = unique_elems;
}
void ExpressionAnalyzer::makeSetsForIndex()
{
if (storage && ast && storage->supportsIndexForIn())
@ -1306,6 +1343,7 @@ void ExpressionAnalyzer::getAggregates(ASTPtr ast, ExpressionActionsPtr & action
AggregateDescription aggregate;
aggregate.column_name = node->getColumnName();
/// Агрегатные функции уникализируются.
for (size_t i = 0; i < aggregate_descriptions.size(); ++i)
if (aggregate_descriptions[i].column_name == aggregate.column_name)
return;
@ -1711,7 +1749,8 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates)
{
for (NamesAndTypesList::iterator it = aggregation_keys.begin(); it != aggregation_keys.end(); ++it)
key_names.push_back(it->name);
key_names.emplace_back(it->name);
aggregates = aggregate_descriptions;
}

View File

@ -4,136 +4,49 @@
#include <DB/Dictionaries/IDictionarySource.h>
#include <statdaemons/ext/scope_guard.hpp>
#include <Poco/Util/Application.h>
#include <Poco/Glob.h>
namespace DB
{
namespace
{
std::string getDictionariesConfigPath(const Poco::Util::AbstractConfiguration & config)
std::set<std::string> getDictionariesConfigPaths(const Poco::Util::AbstractConfiguration & config)
{
const auto path = config.getString("dictionaries_config", "");
if (path.empty())
return path;
auto pattern = config.getString("dictionaries_config", "");
if (pattern.empty())
return {};
if (path[0] != '/')
std::set<std::string> files;
if (pattern[0] != '/')
{
const auto app_config_path = config.getString("config-file", "config.xml");
const auto config_dir = Poco::Path{app_config_path}.parent().toString();
const auto absolute_path = config_dir + path;
if (Poco::File{absolute_path}.exists())
return absolute_path;
const auto absolute_path = config_dir + pattern;
Poco::Glob::glob(absolute_path, files, 0);
if (!files.empty())
return files;
}
return path;
Poco::Glob::glob(pattern, files, 0);
return files;
}
}
void ExternalDictionaries::reloadImpl()
{
const auto config_path = getDictionariesConfigPath(Poco::Util::Application::instance().config());
const Poco::File config_file{config_path};
const auto config_paths = getDictionariesConfigPaths(Poco::Util::Application::instance().config());
if (config_path.empty() || !config_file.exists())
{
LOG_WARNING(log, "config file '" + config_path + "' does not exist");
}
else
{
const auto last_modified = config_file.getLastModified();
if (last_modified > config_last_modified)
{
stored_exceptions.clear();
/// definitions of dictionaries may have changed, recreate all of them
config_last_modified = last_modified;
const auto config = new Poco::Util::XMLConfiguration{config_path};
SCOPE_EXIT(
config->release();
);
/// get all dictionaries' definitions
Poco::Util::AbstractConfiguration::Keys keys;
config->keys(keys);
/// for each dictionary defined in xml config
for (const auto & key : keys)
{
std::string name;
try
{
if (0 != strncmp(key.data(), "dictionary", strlen("dictionary")))
{
LOG_WARNING(log, "unknown node in dictionaries file: '" + key + "', 'dictionary'");
continue;
}
name = config->getString(key + ".name");
if (name.empty())
{
LOG_WARNING(log, "dictionary name cannot be empty");
continue;
}
auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context);
if (!dict_ptr->isCached())
{
const auto & lifetime = dict_ptr->getLifetime();
if (lifetime.min_sec != 0 && lifetime.max_sec != 0)
{
std::uniform_int_distribution<std::uint64_t> distribution{
lifetime.min_sec,
lifetime.max_sec
};
update_times[name] = std::chrono::system_clock::now() +
std::chrono::seconds{distribution(rnd_engine)};
}
}
auto it = dictionaries.find(name);
/// add new dictionary or update an existing version
if (it == std::end(dictionaries))
{
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
dictionaries.emplace(name, std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release()));
}
else
it->second->set(dict_ptr.release());
}
catch (...)
{
if (!name.empty())
stored_exceptions.emplace(name, std::current_exception());
try
{
throw;
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually. " << e.displayText());
}
catch (const std::exception & e)
{
LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually. " << e.what());
}
catch (...)
{
LOG_ERROR(log, "Cannot load external dictionary! You must resolve this manually.");
}
}
}
}
}
for (const auto & config_path : config_paths)
reloadFromFile(config_path);
/// periodic update
for (auto & dictionary : dictionaries)
{
try
{
auto current = dictionary.second->get();
auto current = dictionary.second.first->get();
const auto & lifetime = current->getLifetime();
/// do not update dictionaries with zero as lifetime
@ -160,7 +73,7 @@ void ExternalDictionaries::reloadImpl()
{
/// create new version of dictionary
auto new_version = current->clone();
dictionary.second->set(new_version.release());
dictionary.second.first->set(new_version.release());
}
}
}
@ -171,7 +84,7 @@ void ExternalDictionaries::reloadImpl()
}
catch (const std::exception & e)
{
LOG_ERROR(log, "Cannot load external dictionary '" << dictionary.first
LOG_ERROR(log, "Cannot update external dictionary '" << dictionary.first
<< "'! You must resolve this manually. " << e.what());
}
catch (...)
@ -182,4 +95,116 @@ void ExternalDictionaries::reloadImpl()
}
}
void ExternalDictionaries::reloadFromFile(const std::string & config_path)
{
const Poco::File config_file{config_path};
if (config_path.empty() || !config_file.exists())
{
LOG_WARNING(log, "config file '" + config_path + "' does not exist");
}
else
{
auto it = last_modification_times.find(config_path);
if (it == std::end(last_modification_times))
it = last_modification_times.emplace(config_path, Poco::Timestamp{0}).first;
auto & config_last_modified = it->second;
const auto last_modified = config_file.getLastModified();
if (last_modified > config_last_modified)
{
stored_exceptions.clear();
/// definitions of dictionaries may have changed, recreate all of them
config_last_modified = last_modified;
const auto config = new Poco::Util::XMLConfiguration{config_path};
SCOPE_EXIT(config->release());
/// get all dictionaries' definitions
Poco::Util::AbstractConfiguration::Keys keys;
config->keys(keys);
/// for each dictionary defined in xml config
for (const auto & key : keys)
{
std::string name;
if (0 != strncmp(key.data(), "dictionary", strlen("dictionary")))
{
LOG_WARNING(log, config_path << ": unknown node in dictionaries file: '" << key + "', 'dictionary'");
continue;
}
try
{
name = config->getString(key + ".name");
if (name.empty())
{
LOG_WARNING(log, config_path << ": dictionary name cannot be empty");
continue;
}
auto it = dictionaries.find(name);
if (it != std::end(dictionaries))
if (it->second.second != config_path)
throw std::runtime_error{"Overriding dictionary from file " + it->second.second};
auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context);
if (!dict_ptr->isCached())
{
const auto & lifetime = dict_ptr->getLifetime();
if (lifetime.min_sec != 0 && lifetime.max_sec != 0)
{
std::uniform_int_distribution<std::uint64_t> distribution{
lifetime.min_sec,
lifetime.max_sec
};
update_times[name] = std::chrono::system_clock::now() +
std::chrono::seconds{distribution(rnd_engine)};
}
}
/// add new dictionary or update an existing version
if (it == std::end(dictionaries))
{
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
dictionaries.emplace(name, dictionary_origin_pair_t{
std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release()),
config_path
});
}
else
it->second.first->set(dict_ptr.release());
}
catch (...)
{
if (!name.empty())
stored_exceptions.emplace(name, std::current_exception());
try
{
throw;
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name
<< "'! You must resolve this manually. " << e.displayText());
}
catch (const std::exception & e)
{
LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name
<< "'! You must resolve this manually. " << e.what());
}
catch (...)
{
LOG_ERROR(log, config_path << ": cannot create external dictionary '" << name
<< "'! You must resolve this manually.");
}
}
}
}
}
}
}

View File

@ -664,8 +664,15 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu
* то в качестве размера блока будем использовать limit + offset (чтобы не читать из таблицы больше, чем запрошено),
* а также установим количество потоков в 1 и отменим асинхронное выполнение конвейера запроса.
*/
if (!query.distinct && !query.prewhere_expression && !query.where_expression && !query.group_expression_list && !query.having_expression && !query.order_expression_list
&& query.limit_length && !query_analyzer->hasAggregation() && limit_length + limit_offset < settings.max_block_size)
if (!query.distinct
&& !query.prewhere_expression
&& !query.where_expression
&& !query.group_expression_list
&& !query.having_expression
&& !query.order_expression_list
&& query.limit_length
&& !query_analyzer->hasAggregation()
&& limit_length + limit_offset < settings.max_block_size)
{
settings.max_block_size = limit_length + limit_offset;
settings.max_threads = 1;
@ -678,12 +685,6 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu
/// Инициализируем изначальные потоки данных, на которые накладываются преобразования запроса. Таблица или подзапрос?
if (!interpreter_subquery)
{
/** При распределённой обработке запроса, на все удалённые серверы отправляются временные таблицы,
* полученные из глобальных подзапросов - GLOBAL IN/JOIN.
*/
if (storage && storage->isRemote())
storage->storeExternalTables(query_analyzer->getExternalTables());
streams = storage->read(required_columns, query_ptr,
context, settings_for_storage, from_stage,
settings.max_block_size, settings.max_threads);
@ -817,12 +818,10 @@ static SortDescription getSortDescription(ASTSelectQuery & query)
{
SortDescription order_descr;
order_descr.reserve(query.order_expression_list->children.size());
for (ASTs::iterator it = query.order_expression_list->children.begin();
it != query.order_expression_list->children.end();
++it)
for (const auto & elem : query.order_expression_list->children)
{
String name = (*it)->children.front()->getColumnName();
const ASTOrderByElement & order_by_elem = typeid_cast<const ASTOrderByElement &>(**it);
String name = elem->children.front()->getColumnName();
const ASTOrderByElement & order_by_elem = typeid_cast<const ASTOrderByElement &>(*elem);
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.collator);
}

View File

@ -170,11 +170,20 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
ConstColumnPlainPtrs key_columns(keys_size);
data_types.resize(keys_size);
/// Константные столбцы справа от IN поддерживается не напрямую. Для этого, они сначала материализуется.
Columns materialized_columns;
/// Запоминаем столбцы, с которыми будем работать
for (size_t i = 0; i < keys_size; ++i)
{
key_columns[i] = block.getByPosition(i).column;
data_types[i] = block.getByPosition(i).type;
if (key_columns[i]->isConst())
{
materialized_columns.emplace_back(static_cast<const IColumnConst *>(key_columns[i])->convertToFullColumn());
key_columns[i] = materialized_columns.back().get();
}
}
size_t rows = block.rows();

View File

@ -83,7 +83,9 @@ void executeQuery(
String query(begin, query_size);
LOG_DEBUG(&Logger::get("executeQuery"), query);
String logged_query = query;
std::replace(logged_query.begin(), logged_query.end(), '\n', ' ');
LOG_DEBUG(&Logger::get("executeQuery"), logged_query);
/// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем.
ProcessList::EntryPtr process_list_entry;

View File

@ -62,9 +62,9 @@ int main(int argc, char ** argv)
DB::BlockInputStreamPtr stream = new DB::OneBlockInputStream(block);
DB::AggregatedDataVariants aggregated_data_variants;
DB::ColumnNumbers key_column_numbers;
key_column_numbers.push_back(0);
key_column_numbers.push_back(1);
DB::Names key_column_names;
key_column_names.emplace_back("x");
key_column_names.emplace_back("s1");
DB::AggregateFunctionFactory factory;
@ -73,7 +73,7 @@ int main(int argc, char ** argv)
DB::DataTypes empty_list_of_types;
aggregate_descriptions[0].function = factory.get("count", empty_list_of_types);
DB::Aggregator aggregator(key_column_numbers, aggregate_descriptions, false, 0, DB::OverflowMode::THROW, nullptr, 0, 0);
DB::Aggregator aggregator(key_column_names, aggregate_descriptions, false, 0, DB::OverflowMode::THROW, nullptr, 0, 0);
{
Poco::Stopwatch stopwatch;

View File

@ -247,8 +247,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
catch (Exception & e)
{
std::stringstream s;
s << "Code: " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
s << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
<< ", Stack trace:\n\n" << e.getStackTrace().toString();
LOG_ERROR(log, s.str());
trySendExceptionToClient(s, request, response, used_output);
}
@ -257,18 +257,21 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
std::stringstream s;
s << "Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
LOG_ERROR(log, s.str());
trySendExceptionToClient(s, request, response, used_output);
}
catch (std::exception & e)
{
std::stringstream s;
s << "Code: " << ErrorCodes::STD_EXCEPTION << ". " << e.what();
LOG_ERROR(log, s.str());
trySendExceptionToClient(s, request, response, used_output);
}
catch (...)
{
std::stringstream s;
s << "Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ". Unknown exception.";
LOG_ERROR(log, s.str());
trySendExceptionToClient(s, request, response, used_output);
}
}

View File

@ -716,7 +716,7 @@ void TCPHandler::run()
}
catch (Exception & e)
{
LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText()
LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
}
catch (Poco::Exception & e)

View File

@ -44,7 +44,7 @@ MergeTreeData::MergeTreeData(
date_column_name(date_column_name_), sampling_expression(sampling_expression_),
index_granularity(index_granularity_),
mode(mode_), sign_column(sign_column_), columns_to_sum(columns_to_sum_),
settings(settings_), primary_expr_ast(primary_expr_ast_->clone()),
settings(settings_), primary_expr_ast(primary_expr_ast_ ? primary_expr_ast_->clone() : nullptr),
require_part_metadata(require_part_metadata_),
full_path(full_path_), columns(columns_),
broken_part_callback(broken_part_callback_),
@ -89,18 +89,23 @@ MergeTreeData::MergeTreeData(
Poco::File(full_path).createDirectories();
Poco::File(full_path + "detached").createDirectory();
/// инициализируем описание сортировки
sort_descr.reserve(primary_expr_ast->children.size());
for (const ASTPtr & ast : primary_expr_ast->children)
if (primary_expr_ast)
{
String name = ast->getColumnName();
sort_descr.push_back(SortColumnDescription(name, 1));
/// инициализируем описание сортировки
sort_descr.reserve(primary_expr_ast->children.size());
for (const ASTPtr & ast : primary_expr_ast->children)
{
String name = ast->getColumnName();
sort_descr.push_back(SortColumnDescription(name, 1));
}
primary_expr = ExpressionAnalyzer(primary_expr_ast, context, getColumnsList()).getActions(false);
ExpressionActionsPtr projected_expr = ExpressionAnalyzer(primary_expr_ast, context, getColumnsList()).getActions(true);
primary_key_sample = projected_expr->getSampleBlock();
}
primary_expr = ExpressionAnalyzer(primary_expr_ast, context, getColumnsList()).getActions(false);
ExpressionActionsPtr projected_expr = ExpressionAnalyzer(primary_expr_ast, context, getColumnsList()).getActions(true);
primary_key_sample = projected_expr->getSampleBlock();
else if (mode != Unsorted)
throw Exception("Primary key could be empty only for UnsortedMergeTree", ErrorCodes::BAD_ARGUMENTS);
}
UInt64 MergeTreeData::getMaxDataPartIndex()
@ -120,6 +125,7 @@ std::string MergeTreeData::getModePrefix() const
case Collapsing: return "Collapsing";
case Summing: return "Summing";
case Aggregating: return "Aggregating";
case Unsorted: return "Unsorted";
default:
throw Exception("Unknown mode of operation for MergeTreeData: " + toString(mode), ErrorCodes::LOGICAL_ERROR);
@ -386,8 +392,14 @@ void MergeTreeData::checkAlter(const AlterCommands & params)
/// Список столбцов, которые нельзя трогать.
/// sampling_expression можно не учитывать, потому что он обязан содержаться в первичном ключе.
Names keys = primary_expr->getRequiredColumns();
Names keys;
if (primary_expr)
keys = primary_expr->getRequiredColumns();
keys.push_back(sign_column);
std::sort(keys.begin(), keys.end());
for (const AlterCommand & command : params)
@ -510,7 +522,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
BlockInputStreamPtr part_in = new MergeTreeBlockInputStream(full_path + part->name + '/',
DEFAULT_MERGE_BLOCK_SIZE, expression->getRequiredColumns(), *this, part, ranges, false, nullptr, "", false);
ExpressionBlockInputStream in(part_in, expression);
MergedColumnOnlyOutputStream out(*this, full_path + part->name + '/', true);
MergedColumnOnlyOutputStream out(*this, full_path + part->name + '/', true, CompressionMethod::LZ4);
in.readPrefix();
out.writePrefix();

View File

@ -8,6 +8,7 @@
#include <DB/DataStreams/SummingSortedBlockInputStream.h>
#include <DB/DataStreams/AggregatingSortedBlockInputStream.h>
#include <DB/DataStreams/MaterializingBlockInputStream.h>
#include <DB/DataStreams/ConcatBlockInputStream.h>
namespace DB
@ -335,8 +336,12 @@ MergeTreeData::DataPartPtr MergeTreeDataMerger::mergeParts(
__sync_add_and_fetch(&merge_entry->bytes_read_uncompressed, value.bytes);
});
src_streams.push_back(new MaterializingBlockInputStream{
new ExpressionBlockInputStream(input.release(), data.getPrimaryExpression())});
if (data.mode != MergeTreeData::Unsorted)
src_streams.push_back(new MaterializingBlockInputStream{
new ExpressionBlockInputStream(input.release(), data.getPrimaryExpression())});
else
src_streams.push_back(input.release());
sum_rows_approx += parts[i]->size * data.index_granularity;
}
@ -363,13 +368,17 @@ MergeTreeData::DataPartPtr MergeTreeDataMerger::mergeParts(
merged_stream = std::make_unique<AggregatingSortedBlockInputStream>(src_streams, data.getSortDescription(), DEFAULT_MERGE_BLOCK_SIZE);
break;
case MergeTreeData::Unsorted:
merged_stream = std::make_unique<ConcatBlockInputStream>(src_streams);
break;
default:
throw Exception("Unknown mode of operation for MergeTreeData: " + toString(data.mode), ErrorCodes::LOGICAL_ERROR);
}
const String new_part_tmp_path = data.getFullPath() + "tmp_" + merged_name + "/";
MergedBlockOutputStream to{data, new_part_tmp_path, union_columns};
MergedBlockOutputStream to{data, new_part_tmp_path, union_columns, CompressionMethod::LZ4};
merged_stream->readPrefix();
to.writePrefix();

View File

@ -83,6 +83,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
PKCondition key_condition(query, context, data.getColumnsList(), data.getSortDescription());
PKCondition date_condition(query, context, data.getColumnsList(), SortDescription(1, SortColumnDescription(data.date_column_name, 1)));
if (settings.force_index_by_date && date_condition.alwaysTrue())
throw Exception("Index by date is not used and setting 'force_index_by_date' is set.", ErrorCodes::INDEX_NOT_USED);
/// Выберем куски, в которых могут быть данные, удовлетворяющие date_condition, и которые подходят под условие на _part.
{
auto prev_parts = parts;
@ -268,7 +271,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
for (auto & part : parts)
{
RangesInDataPart ranges(part, (*part_index)++);
ranges.ranges = markRangesFromPkRange(part->index, key_condition, settings);
if (data.mode != MergeTreeData::Unsorted)
ranges.ranges = markRangesFromPkRange(part->index, key_condition, settings);
else
ranges.ranges = MarkRanges{MarkRange{0, part->size}};
if (!ranges.ranges.empty())
{
@ -281,7 +288,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
}
LOG_DEBUG(log, "Selected " << parts.size() << " parts by date, " << parts_with_ranges.size() << " parts by key, "
<< sum_marks << " marks to read from " << sum_ranges << " ranges");
<< sum_marks << " marks to read from " << sum_ranges << " ranges");
BlockInputStreams res;

View File

@ -94,15 +94,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa
new_data_part->is_temp = true;
/// Если для сортировки надо вычислить некоторые столбцы - делаем это.
data.getPrimaryExpression()->execute(block);
if (data.mode != MergeTreeData::Unsorted)
data.getPrimaryExpression()->execute(block);
SortDescription sort_descr = data.getSortDescription();
/// Сортируем.
stableSortBlock(block, sort_descr);
if (data.mode != MergeTreeData::Unsorted)
stableSortBlock(block, sort_descr);
NamesAndTypesList columns = data.getColumnsList().filter(block.getColumnsList().getNames());
MergedBlockOutputStream out(data, part_tmp_path, columns);
MergedBlockOutputStream out(data, part_tmp_path, columns, CompressionMethod::LZ4);
out.getIndex().reserve(part_size * sort_descr.size());
out.writePrefix();

View File

@ -21,10 +21,7 @@
namespace DB
{
const int SLEEP_AFTER_MERGE = 1;
const int SLEEP_NO_WORK = 10;
const int SLEEP_AFTER_ERROR = 60;
const int NOTHING_TO_MERGE_PERIOD = 10;
StorageChunkMerger::TableNames StorageChunkMerger::currently_written_groups;
@ -226,6 +223,35 @@ Block StorageChunkMerger::getBlockWithVirtualColumns(const Storages & selected_t
return res;
}
class StorageChunkMerger::MergeTask
{
public:
MergeTask(const StorageChunkMerger & chunk_merger_, DB::Context & context_, Logger * log_)
:
shutdown_called(false),
chunk_merger(chunk_merger_),
context(context_),
log(log_)
{
}
bool merge();
public:
std::atomic<bool> shutdown_called;
private:
bool maybeMergeSomething();
Storages selectChunksToMerge();
bool mergeChunks(const Storages & chunks);
const StorageChunkMerger & chunk_merger;
DB::Context & context;
Logger * log;
time_t last_nothing_to_merge_time = 0;
};
StorageChunkMerger::StorageChunkMerger(
const std::string & this_database_,
const std::string & name_,
@ -242,20 +268,28 @@ StorageChunkMerger::StorageChunkMerger(
this_database(this_database_), name(name_), columns(columns_), source_database(source_database_),
table_name_regexp(table_name_regexp_), destination_name_prefix(destination_name_prefix_), chunks_to_merge(chunks_to_merge_),
context(context_), settings(context.getSettings()),
log(&Logger::get("StorageChunkMerger")), shutdown_called(false)
log(&Logger::get("StorageChunkMerger")),
merge_task(std::make_shared<MergeTask>(*this, context, log))
{
merge_thread = std::thread([this] { mergeThread(); });
_table_column_name = "_table" + VirtualColumnUtils::chooseSuffix(getColumnsList(), "_table");
auto & backgroud_pool = context.getBackgroundPool();
MergeTaskPtr tmp_merge_task = merge_task;
DB::BackgroundProcessingPool::Task task = [tmp_merge_task](BackgroundProcessingPool::Context & pool_context) -> bool
{
return tmp_merge_task->merge();
};
merge_task_handle = backgroud_pool.addTask(task);
}
void StorageChunkMerger::shutdown()
{
if (shutdown_called)
if (merge_task->shutdown_called)
return;
shutdown_called = true;
cancel_merge_thread.set();
merge_thread.join();
merge_task->shutdown_called.store(true);
context.getBackgroundPool().removeTask(merge_task_handle);
}
StorageChunkMerger::~StorageChunkMerger()
@ -263,41 +297,44 @@ StorageChunkMerger::~StorageChunkMerger()
shutdown();
}
void StorageChunkMerger::mergeThread()
bool StorageChunkMerger::MergeTask::merge()
{
while (true)
time_t now = time(0);
if (last_nothing_to_merge_time + NOTHING_TO_MERGE_PERIOD > now)
return false;
if (shutdown_called)
return false;
bool merged = false;
try
{
bool merged = false;
bool error = true;
try
{
merged = maybeMergeSomething();
error = false;
}
catch (const Exception & e)
{
LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
}
catch (const std::exception & e)
{
LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", e.what() = " << e.what());
}
catch (...)
{
LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION);
}
unsigned sleep_ammount = error ? SLEEP_AFTER_ERROR : (merged ? SLEEP_AFTER_MERGE : SLEEP_NO_WORK);
if (shutdown_called || cancel_merge_thread.tryWait(1000 * sleep_ammount))
break;
merged = maybeMergeSomething();
}
catch (const Exception & e)
{
LOG_ERROR(log, "StorageChunkMerger at " << chunk_merger.this_database << "." << chunk_merger.name << " failed to merge: Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
}
catch (const Poco::Exception & e)
{
LOG_ERROR(log, "StorageChunkMerger at " << chunk_merger.this_database << "." << chunk_merger.name << " failed to merge: Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
}
catch (const std::exception & e)
{
LOG_ERROR(log, "StorageChunkMerger at " << chunk_merger.this_database << "." << chunk_merger.name << " failed to merge: std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", e.what() = " << e.what());
}
catch (...)
{
LOG_ERROR(log, "StorageChunkMerger at " << chunk_merger.this_database << "." << chunk_merger.name << " failed to merge: unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION);
}
if (!merged)
last_nothing_to_merge_time = now;
return merged;
}
static std::string makeName(const std::string & prefix, const std::string & first_chunk, const std::string & last_chunk)
@ -308,7 +345,7 @@ static std::string makeName(const std::string & prefix, const std::string & firs
return prefix + first_chunk + "_" + last_chunk.substr(lcp);
}
bool StorageChunkMerger::maybeMergeSomething()
bool StorageChunkMerger::MergeTask::maybeMergeSomething()
{
Storages chunks = selectChunksToMerge();
if (chunks.empty() || shutdown_called)
@ -316,7 +353,7 @@ bool StorageChunkMerger::maybeMergeSomething()
return mergeChunks(chunks);
}
StorageChunkMerger::Storages StorageChunkMerger::selectChunksToMerge()
StorageChunkMerger::Storages StorageChunkMerger::MergeTask::selectChunksToMerge()
{
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
@ -324,26 +361,26 @@ StorageChunkMerger::Storages StorageChunkMerger::selectChunksToMerge()
Databases & databases = context.getDatabases();
if (!databases.count(source_database))
throw Exception("No database " + source_database, ErrorCodes::UNKNOWN_DATABASE);
if (!databases.count(chunk_merger.source_database))
throw Exception("No database " + chunk_merger.source_database, ErrorCodes::UNKNOWN_DATABASE);
Tables & tables = databases[source_database];
Tables & tables = databases[chunk_merger.source_database];
for (Tables::iterator it = tables.begin(); it != tables.end(); ++it)
{
StoragePtr table = it->second;
if (table_name_regexp.match(it->first) &&
if (chunk_merger.table_name_regexp.match(it->first) &&
!typeid_cast<StorageChunks *>(&*table) &&
!typeid_cast<StorageChunkMerger *>(&*table) &&
!typeid_cast<StorageChunkRef *>(&*table))
{
res.push_back(table);
if (res.size() >= chunks_to_merge)
if (res.size() >= chunk_merger.chunks_to_merge)
break;
}
}
if (res.size() < chunks_to_merge)
if (res.size() < chunk_merger.chunks_to_merge)
res.clear();
return res;
@ -357,7 +394,7 @@ static ASTPtr newIdentifier(const std::string & name, ASTIdentifier::Kind kind)
return res;
}
bool StorageChunkMerger::mergeChunks(const Storages & chunks)
bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
{
typedef std::map<std::string, DataTypePtr> ColumnsMap;
@ -371,11 +408,11 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
/// Объединим множества столбцов сливаемых чанков.
ColumnsMap known_columns_types;
for (const NameAndTypePair & column : getColumnsList())
for (const NameAndTypePair & column : chunk_merger.getColumnsList())
known_columns_types.insert(std::make_pair(column.name, column.type));
NamesAndTypesListPtr required_columns = new NamesAndTypesList;
*required_columns = *columns;
*required_columns = *chunk_merger.columns;
for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index)
{
@ -402,8 +439,8 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
std::string formatted_columns = formatColumnsForCreateQuery(*required_columns);
std::string new_table_name = makeName(destination_name_prefix, chunks.front()->getTableName(), chunks.back()->getTableName());
std::string new_table_full_name = backQuoteIfNeed(source_database) + "." + backQuoteIfNeed(new_table_name);
std::string new_table_name = makeName(chunk_merger.destination_name_prefix, chunks.front()->getTableName(), chunks.back()->getTableName());
std::string new_table_full_name = backQuoteIfNeed(chunk_merger.source_database) + "." + backQuoteIfNeed(new_table_name);
StoragePtr new_storage_ptr;
try
@ -411,8 +448,8 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
{
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
if (!context.getDatabases().count(source_database))
throw Exception("Destination database " + source_database + " for table " + name + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
if (!context.getDatabases().count(chunk_merger.source_database))
throw Exception("Destination database " + chunk_merger.source_database + " for table " + chunk_merger.name + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
LOG_TRACE(log, "Will merge " << chunks.size() << " chunks: from " << chunks[0]->getTableName() << " to " << chunks.back()->getTableName() << " to new table " << new_table_name << ".");
@ -431,7 +468,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
/// Выполним запрос для создания Chunks таблицы.
executeQuery("CREATE TABLE " + new_table_full_name + " " + formatted_columns + " ENGINE = Chunks", context, true);
new_storage_ptr = context.getTable(source_database, new_table_name);
new_storage_ptr = context.getTable(chunk_merger.source_database, new_table_name);
/// Скопируем данные в новую таблицу.
StorageChunks & new_storage = typeid_cast<StorageChunks &>(*new_storage_ptr);
@ -451,7 +488,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
ASTPtr select_expression_list;
ASTPtr database;
ASTPtr table; /// Идентификатор или подзапрос (рекурсивно ASTSelectQuery)
select_query->database = newIdentifier(source_database, ASTIdentifier::Database);
select_query->database = newIdentifier(chunk_merger.source_database, ASTIdentifier::Database);
select_query->table = newIdentifier(src_storage->getTableName(), ASTIdentifier::Table);
ASTExpressionList * select_list = new ASTExpressionList;
select_query->select_expression_list = select_list;
@ -467,7 +504,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
src_column_names,
select_query_ptr,
context,
settings,
chunk_merger.settings,
processed_stage,
DEFAULT_MERGE_BLOCK_SIZE);
@ -502,7 +539,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
/// Если БД успели удалить, ничего не делаем.
if (context.getDatabases().count(source_database))
if (context.getDatabases().count(chunk_merger.source_database))
{
for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index)
{
@ -510,16 +547,16 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
std::string src_name = src_storage->getTableName();
/// Если таблицу успели удалить, ничего не делаем.
if (!context.isTableExist(source_database, src_name))
if (!context.isTableExist(chunk_merger.source_database, src_name))
continue;
/// Отцепляем исходную таблицу. Ее данные и метаданные остаются на диске.
tables_to_drop.push_back(context.detachTable(source_database, src_name));
tables_to_drop.push_back(context.detachTable(chunk_merger.source_database, src_name));
/// Создаем на ее месте ChunkRef. Это возможно только потому что у ChunkRef нет ни, ни метаданных.
try
{
context.addTable(source_database, src_name, StorageChunkRef::create(src_name, context, source_database, new_table_name, false));
context.addTable(chunk_merger.source_database, src_name, StorageChunkRef::create(src_name, context, chunk_merger.source_database, new_table_name, false));
}
catch (...)
{
@ -537,7 +574,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks)
table_locks.clear();
for (StoragePtr table : tables_to_drop)
{
InterpreterDropQuery::dropDetachedTable(source_database, table, context);
InterpreterDropQuery::dropDetachedTable(chunk_merger.source_database, table, context);
/// NOTE: Если между подменой таблицы и этой строчкой кто-то успеет попытаться создать новую таблицу на ее месте,
/// что-нибудь может сломаться.
}

View File

@ -166,6 +166,8 @@ BlockInputStreams StorageDistributed::read(
if (settings.limits.max_network_bandwidth)
throttler.reset(new Throttler(settings.limits.max_network_bandwidth));
Tables external_tables = context.getExternalTables();
/// Цикл по шардам.
for (auto & conn_pool : cluster.pools)
res.emplace_back(new RemoteBlockInputStream{
@ -177,9 +179,6 @@ BlockInputStreams StorageDistributed::read(
{
DB::Context new_context = context;
new_context.setSettings(new_settings);
for (auto & it : external_tables)
if (!new_context.tryGetExternalTable(it.first))
new_context.addExternalTable(it.first, it.second);
for (size_t i = 0; i < cluster.getLocalNodesNum(); ++i)
{
@ -193,7 +192,6 @@ BlockInputStreams StorageDistributed::read(
}
}
external_tables.clear();
return res;
}

View File

@ -350,7 +350,7 @@ StoragePtr StorageFactory::get(
}
else if (endsWith(name, "MergeTree"))
{
/** Движки [Replicated][Summing|Collapsing|Aggregating|]MergeTree (8 комбинаций)
/** Движки [Replicated][|Summing|Collapsing|Aggregating|Unsorted]MergeTree (2 * 5 комбинаций)
* В качестве аргумента для движка должно быть указано:
* - (для Replicated) Путь к таблице в ZooKeeper
* - (для Replicated) Имя реплики в ZooKeeper
@ -367,6 +367,7 @@ StoragePtr StorageFactory::get(
* CollapsingMergeTree(date, [sample_key], primary_key, index_granularity, sign)
* SummingMergeTree(date, [sample_key], primary_key, index_granularity, [columns_to_sum])
* AggregatingMergeTree(date, [sample_key], primary_key, index_granularity)
* UnsortedMergeTree(date, index_granularity) TODO Добавить описание ниже.
*/
const char * verbose_help = R"(
@ -445,6 +446,8 @@ For further info please read the documentation: http://clickhouse.yandex-team.ru
mode = MergeTreeData::Summing;
else if (name_part == "Aggregating")
mode = MergeTreeData::Aggregating;
else if (name_part == "Unsorted")
mode = MergeTreeData::Unsorted;
else if (!name_part.empty())
throw Exception("Unknown storage " + name + verbose_help, ErrorCodes::UNKNOWN_STORAGE);
@ -458,7 +461,26 @@ For further info please read the documentation: http://clickhouse.yandex-team.ru
/// NOTE Слегка запутанно.
size_t num_additional_params = (replicated ? 2 : 0) + (mode == MergeTreeData::Collapsing);
if (mode != MergeTreeData::Summing
if (mode == MergeTreeData::Unsorted
&& args.size() != num_additional_params + 2)
{
String params;
if (replicated)
params +=
"\npath in ZooKeeper,"
"\nreplica name,";
params +=
"\nname of column with date,"
"\nindex granularity\n";
throw Exception("Storage " + name + " requires "
+ toString(num_additional_params + 2) + " parameters: " + params + verbose_help,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
if (mode != MergeTreeData::Summing && mode != MergeTreeData::Unsorted
&& args.size() != num_additional_params + 3
&& args.size() != num_additional_params + 4)
{
@ -579,9 +601,10 @@ For further info please read the documentation: http://clickhouse.yandex-team.ru
else
throw Exception(String("Date column name must be an unquoted string") + verbose_help, ErrorCodes::BAD_ARGUMENTS);
primary_expr_list = extractPrimaryKey(args[1]);
if (mode != MergeTreeData::Unsorted)
primary_expr_list = extractPrimaryKey(args[1]);
auto ast = typeid_cast<ASTLiteral *>(&*args[2]);
auto ast = typeid_cast<ASTLiteral *>(&*args.back());
if (ast && ast->value.getType() == Field::Types::UInt64)
index_granularity = safeGet<UInt64>(ast->value);
else

View File

@ -1058,28 +1058,6 @@ void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTr
}
LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << ".");
if (unreplicated_data)
{
std::lock_guard<std::mutex> unreplicated_lock(unreplicated_mutex);
removed_parts = 0;
parts = unreplicated_data->getDataParts();
for (const auto & part : parts)
{
if (!ActiveDataPartSet::contains(entry.new_part_name, part->name))
continue;
LOG_DEBUG(log, "Removing unreplicated part " << part->name);
++removed_parts;
if (entry.detach)
unreplicated_data->renameAndDetachPart(part, "");
else
unreplicated_data->replaceParts({part}, {}, false);
}
LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << " (in unreplicated data).");
}
}

View File

@ -12,6 +12,7 @@ fi
# Создадим директории для данных второго сервера.
PATH2=/tmp/clickhouse/
[ -d "$PATH2" ] && rm -rf $PATH2
mkdir -p ${PATH2}{data,metadata}/default/
# Создадим второй конфиг с портом 9001.
@ -47,6 +48,7 @@ PID=$!
function finish {
kill $PID
wait
}
trap finish EXIT
@ -104,5 +106,3 @@ $CLIENT1 -n --query="
$CLIENT2 -n --query="
DROP TABLE test.half1;
DROP TABLE test.half2;"
rm -rf $PATH2

View File

@ -0,0 +1,9 @@
\0
\0\0
\0\0\0
\0\0\0\0
\0\0\0\0\0
\0\0\0\0\0\0
\0\0\0\0\0\0\0
\0\0\0\0\0\0\0\0
\0\0\0\0\0\0\0\0\0

View File

@ -0,0 +1,9 @@
SELECT materialize(toFixedString('', 1)) AS x FROM system.one GROUP BY x;
SELECT materialize(toFixedString('', 2)) AS x FROM system.one GROUP BY x;
SELECT materialize(toFixedString('', 3)) AS x FROM system.one GROUP BY x;
SELECT materialize(toFixedString('', 4)) AS x FROM system.one GROUP BY x;
SELECT materialize(toFixedString('', 5)) AS x FROM system.one GROUP BY x;
SELECT materialize(toFixedString('', 6)) AS x FROM system.one GROUP BY x;
SELECT materialize(toFixedString('', 7)) AS x FROM system.one GROUP BY x;
SELECT materialize(toFixedString('', 8)) AS x FROM system.one GROUP BY x;
SELECT materialize(toFixedString('', 9)) AS x FROM system.one GROUP BY x;

View File

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

View File

@ -0,0 +1,3 @@
SET max_rows_to_read = 1000000;
SET read_overflow_mode = 'break';
SELECT concat(toString(number % 256 AS n), '') AS s, n, max(s) FROM system.numbers_mt GROUP BY s, n, n, n, n, n, n, n, n, n ORDER BY s, n;

View File

@ -0,0 +1 @@
SELECT n FROM (SELECT number AS n FROM system.numbers LIMIT 1000000) ORDER BY n, n, n, n, n, n, n, n, n, n LIMIT 1000000, 1;

View File

@ -0,0 +1,30 @@
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0

View File

@ -0,0 +1,31 @@
SELECT 1 IN (SELECT 1);
SELECT materialize(1) IN (SELECT 1);
SELECT 1 IN (SELECT materialize(1));
SELECT materialize(1) IN (SELECT materialize(1));
SELECT (1, 2) IN (SELECT 1, 2);
SELECT (1, materialize(2)) IN (SELECT 1, 2);
SELECT (1, materialize(2)) IN (SELECT materialize(1), 2);
SELECT (1, materialize(2), 'Hello') IN (SELECT materialize(1), 2, 'Hello');
SELECT (1, materialize(2), materialize('Hello')) IN (SELECT materialize(1), 2, 'Hello');
SELECT (1, materialize(2), materialize('Hello')) IN (SELECT materialize(1), 2, materialize('Hello'));
SELECT (1, materialize(2), 'Hello') IN (SELECT materialize(1), 2, materialize('Hello'));
SELECT 'Hello' IN (SELECT 'Hello');
SELECT materialize('Hello') IN (SELECT 'Hello');
SELECT 'Hello' IN (SELECT materialize('Hello'));
SELECT materialize('Hello') IN (SELECT materialize('Hello'));
SELECT 2 IN (SELECT 1);
SELECT materialize(2) IN (SELECT 1);
SELECT 2 IN (SELECT materialize(1));
SELECT materialize(2) IN (SELECT materialize(1));
SELECT (1, 3) IN (SELECT 1, 2);
SELECT (1, materialize(3)) IN (SELECT 1, 2);
SELECT (1, materialize(3)) IN (SELECT materialize(1), 2);
SELECT (1, materialize(2), 'World') IN (SELECT materialize(1), 2, 'Hello');
SELECT (1, materialize(2), materialize('World')) IN (SELECT materialize(1), 2, 'Hello');
SELECT (1, materialize(2), materialize('World')) IN (SELECT materialize(1), 2, materialize('Hello'));
SELECT (1, materialize(2), 'World') IN (SELECT materialize(1), 2, materialize('Hello'));
SELECT 'World' IN (SELECT 'Hello');
SELECT materialize('World') IN (SELECT 'Hello');
SELECT 'World' IN (SELECT materialize('Hello'));
SELECT materialize('World') IN (SELECT materialize('Hello'));

View File

@ -23,7 +23,7 @@ class Null
public:
T data;
bool is_null;
Null() : is_null(true) {}
Null(NullType data) : is_null(true) {}
explicit Null(const T & data_) : data(data_), is_null(false) {}
@ -68,7 +68,7 @@ public:
bool operator== (const NullType other) const { return is_null; }
bool operator!= (const Null<T> & other) const
bool operator!= (const Null<T> & other) const
{
return !(*this == other);
}
@ -82,4 +82,12 @@ public:
};
template<typename T>
T getValueFromNull(const Null<T> & maybe)
{
if (maybe.isNull())
return {};
return maybe;
}
}

View File

@ -0,0 +1,48 @@
#pragma once
#include <zkutil/ZooKeeper.h>
namespace zkutil
{
class Increment
{
public:
Increment(ZooKeeperPtr zk_, const std::string & path_)
: zk(zk_), path(path_)
{
zk->createAncestors(path);
}
size_t get()
{
LOG_TRACE(log, "Get increment");
size_t result = 0;
std::string result_str;
zkutil::Stat stat;
bool success = false;
do
{
if (zk->tryGet(path, result_str, &stat))
{
result = std::stol(result_str) + 1;
success = zk->trySet(path, std::to_string(result), stat.version) == ZOK;
}
else
{
success = zk->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZOK;
}
}
while(!success);
return result;
}
private:
zkutil::ZooKeeperPtr zk;
std::string path;
Logger * log = &Logger::get("zkutil::Increment");
};
}

1
libs/libzstd/README Normal file
View File

@ -0,0 +1 @@
https://github.com/Cyan4973/zstd/tree/765207c54934d478488c236749b01c7d6fc63d70/

View File

@ -0,0 +1,26 @@
ZSTD Library
Copyright (c) 2014-2015, Yann Collet
All rights reserved.
BSD License
Redistribution and use in source and binary forms, with or without modification,
are permitted provided that the following conditions are met:
* Redistributions of source code must retain the above copyright notice, this
list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above copyright notice, this
list of conditions and the following disclaimer in the documentation and/or
other materials provided with the distribution.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,380 @@
/* ******************************************************************
FSE : Finite State Entropy coder
header file
Copyright (C) 2013-2015, Yann Collet.
BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are
met:
* Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above
copyright notice, this list of conditions and the following disclaimer
in the documentation and/or other materials provided with the
distribution.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
You can contact the author at :
- Source repository : https://github.com/Cyan4973/FiniteStateEntropy
- Public forum : https://groups.google.com/forum/#!forum/lz4c
****************************************************************** */
#pragma once
#if defined (__cplusplus)
extern "C" {
#endif
/******************************************
* Includes
******************************************/
#include <stddef.h> // size_t, ptrdiff_t
/******************************************
* FSE simple functions
******************************************/
size_t FSE_compress(void* dst, size_t maxDstSize,
const void* src, size_t srcSize);
size_t FSE_decompress(void* dst, size_t maxDstSize,
const void* cSrc, size_t cSrcSize);
/*
FSE_compress():
Compress content of buffer 'src', of size 'srcSize', into destination buffer 'dst'.
'dst' buffer must be already allocated, and sized to handle worst case situations.
Worst case size evaluation is provided by FSE_compressBound().
return : size of compressed data
Special values : if result == 0, data is uncompressible => Nothing is stored within cSrc !!
if result == 1, data is one constant element x srcSize times. Use RLE compression.
if FSE_isError(result), it's an error code.
FSE_decompress():
Decompress FSE data from buffer 'cSrc', of size 'cSrcSize',
into already allocated destination buffer 'dst', of size 'maxDstSize'.
** Important ** : This function doesn't decompress uncompressed nor RLE data !
return : size of regenerated data (<= maxDstSize)
or an error code, which can be tested using FSE_isError()
*/
size_t FSE_decompressRLE(void* dst, size_t originalSize,
const void* cSrc, size_t cSrcSize);
/*
FSE_decompressRLE():
Decompress specific RLE corner case (equivalent to memset()).
cSrcSize must be == 1. originalSize must be exact.
return : size of regenerated data (==originalSize)
or an error code, which can be tested using FSE_isError()
Note : there is no function provided for uncompressed data, as it's just a simple memcpy()
*/
/******************************************
* Tool functions
******************************************/
size_t FSE_compressBound(size_t size); /* maximum compressed size */
/* Error Management */
unsigned FSE_isError(size_t code); /* tells if a return value is an error code */
const char* FSE_getErrorName(size_t code); /* provides error code string (useful for debugging) */
/******************************************
* FSE advanced functions
******************************************/
/*
FSE_compress2():
Same as FSE_compress(), but allows the selection of 'maxSymbolValue' and 'tableLog'
Both parameters can be defined as '0' to mean : use default value
return : size of compressed data
or -1 if there is an error
*/
size_t FSE_compress2 (void* dst, size_t dstSize, const void* src, size_t srcSize, unsigned maxSymbolValue, unsigned tableLog);
/******************************************
FSE detailed API
******************************************/
/*
int FSE_compress(char* dest, const char* source, int inputSize) does the following:
1. count symbol occurrence from source[] into table count[]
2. normalize counters so that sum(count[]) == Power_of_2 (2^tableLog)
3. save normalized counters to memory buffer using writeHeader()
4. build encoding table 'CTable' from normalized counters
5. encode the data stream using encoding table
int FSE_decompress(char* dest, int originalSize, const char* compressed) performs:
1. read normalized counters with readHeader()
2. build decoding table 'DTable' from normalized counters
3. decode the data stream using decoding table
The following API allows triggering specific sub-functions.
*/
/* *** COMPRESSION *** */
size_t FSE_count(unsigned* count, const unsigned char* src, size_t srcSize, unsigned* maxSymbolValuePtr);
unsigned FSE_optimalTableLog(unsigned tableLog, size_t srcSize, unsigned maxSymbolValue);
size_t FSE_normalizeCount(short* normalizedCounter, unsigned tableLog, const unsigned* count, size_t total, unsigned maxSymbolValue);
size_t FSE_headerBound(unsigned maxSymbolValue, unsigned tableLog);
size_t FSE_writeHeader (void* headerBuffer, size_t headerBufferSize, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
void* FSE_createCTable (unsigned tableLog, unsigned maxSymbolValue);
void FSE_freeCTable (void* CTable);
size_t FSE_buildCTable(void* CTable, const short* normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
size_t FSE_compress_usingCTable (void* dst, size_t dstSize, const void* src, size_t srcSize, const void* CTable);
/*
The first step is to count all symbols. FSE_count() provides one quick way to do this job.
Result will be saved into 'count', a table of unsigned int, which must be already allocated, and have '*maxSymbolValuePtr+1' cells.
'source' is a table of char of size 'sourceSize'. All values within 'src' MUST be <= *maxSymbolValuePtr
*maxSymbolValuePtr will be updated, with its real value (necessarily <= original value)
FSE_count() will return the number of occurrence of the most frequent symbol.
If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
The next step is to normalize the frequencies.
FSE_normalizeCount() will ensure that sum of frequencies is == 2 ^'tableLog'.
It also guarantees a minimum of 1 to any Symbol which frequency is >= 1.
You can use input 'tableLog'==0 to mean "use default tableLog value".
If you are unsure of which tableLog value to use, you can optionally call FSE_optimalTableLog(),
which will provide the optimal valid tableLog given sourceSize, maxSymbolValue, and a user-defined maximum (0 means "default").
The result of FSE_normalizeCount() will be saved into a table,
called 'normalizedCounter', which is a table of signed short.
'normalizedCounter' must be already allocated, and have at least 'maxSymbolValue+1' cells.
The return value is tableLog if everything proceeded as expected.
It is 0 if there is a single symbol within distribution.
If there is an error(typically, invalid tableLog value), the function will return an ErrorCode (which can be tested using FSE_isError()).
'normalizedCounter' can be saved in a compact manner to a memory area using FSE_writeHeader().
'header' buffer must be already allocated.
For guaranteed success, buffer size must be at least FSE_headerBound().
The result of the function is the number of bytes written into 'header'.
If there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()) (for example, buffer size too small).
'normalizedCounter' can then be used to create the compression tables 'CTable'.
The space required by 'CTable' must be already allocated. Its size is provided by FSE_sizeof_CTable().
'CTable' must be aligned of 4 bytes boundaries.
You can then use FSE_buildCTable() to fill 'CTable'.
In both cases, if there is an error, the function will return an ErrorCode (which can be tested using FSE_isError()).
'CTable' can then be used to compress 'source', with FSE_compress_usingCTable().
Similar to FSE_count(), the convention is that 'source' is assumed to be a table of char of size 'sourceSize'
The function returns the size of compressed data (without header), or -1 if failed.
*/
/* *** DECOMPRESSION *** */
size_t FSE_readHeader (short* normalizedCounter, unsigned* maxSymbolValuePtr, unsigned* tableLogPtr, const void* headerBuffer, size_t hbSize);
void* FSE_createDTable(unsigned tableLog);
void FSE_freeDTable(void* DTable);
size_t FSE_buildDTable (void* DTable, const short* const normalizedCounter, unsigned maxSymbolValue, unsigned tableLog);
size_t FSE_decompress_usingDTable(void* dst, size_t maxDstSize, const void* cSrc, size_t cSrcSize, const void* DTable, size_t fastMode);
/*
If the block is RLE compressed, or uncompressed, use the relevant specific functions.
The first step is to obtain the normalized frequencies of symbols.
This can be performed by reading a header with FSE_readHeader().
'normalizedCounter' must be already allocated, and have at least '*maxSymbolValuePtr+1' cells of short.
In practice, that means it's necessary to know 'maxSymbolValue' beforehand,
or size the table to handle worst case situations (typically 256).
FSE_readHeader will provide 'tableLog' and 'maxSymbolValue' stored into the header.
The result of FSE_readHeader() is the number of bytes read from 'header'.
The following values have special meaning :
return 2 : there is only a single symbol value. The value is provided into the second byte of header.
return 1 : data is uncompressed
If there is an error, the function will return an error code, which can be tested using FSE_isError().
The next step is to create the decompression tables 'DTable' from 'normalizedCounter'.
This is performed by the function FSE_buildDTable().
The space required by 'DTable' must be already allocated and properly aligned.
One can create a DTable using FSE_createDTable().
The function will return 1 if DTable is compatible with fastMode, 0 otherwise.
If there is an error, the function will return an error code, which can be tested using FSE_isError().
'DTable' can then be used to decompress 'compressed', with FSE_decompress_usingDTable().
Only trigger fastMode if it was authorized by result of FSE_buildDTable(), otherwise decompression will fail.
cSrcSize must be correct, otherwise decompression will fail.
FSE_decompress_usingDTable() result will tell how many bytes were regenerated.
If there is an error, the function will return an error code, which can be tested using FSE_isError().
*/
/******************************************
* FSE streaming compression API
******************************************/
typedef struct
{
size_t bitContainer;
int bitPos;
char* startPtr;
char* ptr;
} FSE_CStream_t;
typedef struct
{
ptrdiff_t value;
const void* stateTable;
const void* symbolTT;
unsigned stateLog;
} FSE_CState_t;
void FSE_initCStream(FSE_CStream_t* bitC, void* dstBuffer);
void FSE_initCState(FSE_CState_t* CStatePtr, const void* CTable);
void FSE_encodeByte(FSE_CStream_t* bitC, FSE_CState_t* CStatePtr, unsigned char symbol);
void FSE_addBits(FSE_CStream_t* bitC, size_t value, unsigned nbBits);
void FSE_flushBits(FSE_CStream_t* bitC);
void FSE_flushCState(FSE_CStream_t* bitC, const FSE_CState_t* CStatePtr);
size_t FSE_closeCStream(FSE_CStream_t* bitC);
/*
These functions are inner components of FSE_compress_usingCTable().
They allow creation of custom streams, mixing multiple tables and bit sources.
A key property to keep in mind is that encoding and decoding are done **in reverse direction**.
So the first symbol you will encode is the last you will decode, like a lifo stack.
You will need a few variables to track your CStream. They are :
void* CTable; // Provided by FSE_buildCTable()
FSE_CStream_t bitC; // bitStream tracking structure
FSE_CState_t state; // State tracking structure
The first thing to do is to init the bitStream, and the state.
FSE_initCStream(&bitC, dstBuffer);
FSE_initState(&state, CTable);
You can then encode your input data, byte after byte.
FSE_encodeByte() outputs a maximum of 'tableLog' bits at a time.
Remember decoding will be done in reverse direction.
FSE_encodeByte(&bitStream, &state, symbol);
At any time, you can add any bit sequence.
Note : maximum allowed nbBits is 25, for compatibility with 32-bits decoders
FSE_addBits(&bitStream, bitField, nbBits);
The above methods don't commit data to memory, they just store it into local register, for speed.
Local register size is 64-bits on 64-bits systems, 32-bits on 32-bits systems (size_t).
Writing data to memory is a manual operation, performed by the flushBits function.
FSE_flushBits(&bitStream);
Your last FSE encoding operation shall be to flush your last state value(s).
FSE_flushState(&bitStream, &state);
You must then close the bitStream if you opened it with FSE_initCStream().
It's possible to embed some user-info into the header, as an optionalId [0-31].
The function returns the size in bytes of CStream.
If there is an error, it returns an errorCode (which can be tested using FSE_isError()).
size_t size = FSE_closeCStream(&bitStream, optionalId);
*/
/******************************************
* FSE streaming decompression API
******************************************/
//typedef unsigned int bitD_t;
typedef size_t bitD_t;
typedef struct
{
bitD_t bitContainer;
unsigned bitsConsumed;
const char* ptr;
const char* start;
} FSE_DStream_t;
typedef struct
{
bitD_t state;
const void* table;
} FSE_DState_t;
size_t FSE_initDStream(FSE_DStream_t* bitD, const void* srcBuffer, size_t srcSize);
void FSE_initDState(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD, const void* DTable);
unsigned char FSE_decodeSymbol(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD);
bitD_t FSE_readBits(FSE_DStream_t* bitD, unsigned nbBits);
unsigned int FSE_reloadDStream(FSE_DStream_t* bitD);
unsigned FSE_endOfDStream(const FSE_DStream_t* bitD);
unsigned FSE_endOfDState(const FSE_DState_t* DStatePtr);
/*
Let's now decompose FSE_decompress_usingDTable() into its unitary elements.
You will decode FSE-encoded symbols from the bitStream,
and also any other bitFields you put in, **in reverse order**.
You will need a few variables to track your bitStream. They are :
FSE_DStream_t DStream; // Stream context
FSE_DState_t DState; // State context. Multiple ones are possible
const void* DTable; // Decoding table, provided by FSE_buildDTable()
U32 tableLog; // Provided by FSE_readHeader()
The first thing to do is to init the bitStream.
errorCode = FSE_initDStream(&DStream, &optionalId, srcBuffer, srcSize);
You should then retrieve your initial state(s) (multiple ones are possible) :
errorCode = FSE_initDState(&DState, &DStream, DTable, tableLog);
You can then decode your data, symbol after symbol.
For information the maximum number of bits read by FSE_decodeSymbol() is 'tableLog'.
Keep in mind that symbols are decoded in reverse order, like a lifo stack (last in, first out).
unsigned char symbol = FSE_decodeSymbol(&DState, &DStream);
You can retrieve any bitfield you eventually stored into the bitStream (in reverse order)
Note : maximum allowed nbBits is 25
unsigned int bitField = FSE_readBits(&DStream, nbBits);
All above operations only read from local register (which size is controlled by bitD_t==32 bits).
Reading data from memory is manually performed by the reload method.
endSignal = FSE_reloadDStream(&DStream);
FSE_reloadDStream() result tells if there is still some more data to read from DStream.
0 : there is still some data left into the DStream.
1 Dstream reached end of buffer, but is not yet fully extracted. It will not load data from memory any more.
2 Dstream reached its exact end, corresponding in general to decompression completed.
3 Dstream went too far. Decompression result is corrupted.
When reaching end of buffer(1), progress slowly if you decode multiple symbols per loop,
to properly detect the exact end of stream.
After each decoded symbol, check if DStream is fully consumed using this simple test :
FSE_reloadDStream(&DStream) >= 2
When it's done, verify decompression is fully completed, by checking both DStream and the relevant states.
Checking if DStream has reached its end is performed by :
FSE_endOfDStream(&DStream);
Check also the states. There might be some entropy left there, still able to decode some high probability symbol.
FSE_endOfDState(&DState);
*/
#if defined (__cplusplus)
}
#endif

View File

@ -0,0 +1,108 @@
/* ******************************************************************
FSE : Finite State Entropy coder
header file for static linking (only)
Copyright (C) 2013-2015, Yann Collet
BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are
met:
* Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above
copyright notice, this list of conditions and the following disclaimer
in the documentation and/or other materials provided with the
distribution.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
You can contact the author at :
- Source repository : https://github.com/Cyan4973/FiniteStateEntropy
- Public forum : https://groups.google.com/forum/#!forum/lz4c
****************************************************************** */
#pragma once
#if defined (__cplusplus)
extern "C" {
#endif
/******************************************
* Tool functions
******************************************/
#define FSE_MAX_HEADERSIZE 512
#define FSE_COMPRESSBOUND(size) (size + (size>>7) + FSE_MAX_HEADERSIZE) /* Macro can be useful for static allocation */
/******************************************
* Static allocation
******************************************/
/* You can statically allocate a CTable as a table of U32 using below macro */
#define FSE_CTABLE_SIZE_U32(maxTableLog, maxSymbolValue) (1 + (1<<(maxTableLog-1)) + ((maxSymbolValue+1)*2))
#define FSE_DTABLE_SIZE_U32(maxTableLog) ((1<<maxTableLog)+1)
/******************************************
* FSE supported API for DLL
******************************************/
#include "fse.h"
/******************************************
* Error Management
******************************************/
#define FSE_LIST_ERRORS(ITEM) \
ITEM(FSE_OK_NoError) ITEM(FSE_ERROR_GENERIC) \
ITEM(FSE_ERROR_tableLog_tooLarge) ITEM(FSE_ERROR_maxSymbolValue_tooLarge) \
ITEM(FSE_ERROR_dstSize_tooSmall) ITEM(FSE_ERROR_srcSize_wrong)\
ITEM(FSE_ERROR_corruptionDetected) \
ITEM(FSE_ERROR_maxCode)
#define FSE_GENERATE_ENUM(ENUM) ENUM,
typedef enum { FSE_LIST_ERRORS(FSE_GENERATE_ENUM) } FSE_errorCodes; /* enum is exposed, to detect & handle specific errors; compare function result to -enum value */
/******************************************
* FSE advanced API
******************************************/
size_t FSE_countFast(unsigned* count, const unsigned char* src, size_t srcSize, unsigned* maxSymbolValuePtr);
/* same as FSE_count(), but won't check if input really respect that all values within src are <= *maxSymbolValuePtr */
size_t FSE_buildCTable_raw (void* CTable, unsigned nbBits);
/* create a fake CTable, designed to not compress an input where each element uses nbBits */
size_t FSE_buildCTable_rle (void* CTable, unsigned char symbolValue);
/* create a fake CTable, designed to compress a single identical value */
size_t FSE_buildDTable_raw (void* DTable, unsigned nbBits);
/* create a fake DTable, designed to read an uncompressed bitstream where each element uses nbBits */
size_t FSE_buildDTable_rle (void* DTable, unsigned char symbolValue);
/* create a fake DTable, designed to always generate the same symbolValue */
/******************************************
* FSE streaming API
******************************************/
bitD_t FSE_readBitsFast(FSE_DStream_t* bitD, unsigned nbBits);
/* faster, but works only if nbBits >= 1 (otherwise, result will be corrupted) */
unsigned char FSE_decodeSymbolFast(FSE_DState_t* DStatePtr, FSE_DStream_t* bitD);
/* faster, but works only if nbBits >= 1 (otherwise, result will be corrupted) */
#if defined (__cplusplus)
}
#endif

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,93 @@
/*
zstd - standard compression library
Header File
Copyright (C) 2014-2015, Yann Collet.
BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are
met:
* Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above
copyright notice, this list of conditions and the following disclaimer
in the documentation and/or other materials provided with the
distribution.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
You can contact the author at :
- zstd source repository : https://github.com/Cyan4973/zstd
- ztsd public forum : https://groups.google.com/forum/#!forum/lz4c
*/
#pragma once
#if defined (__cplusplus)
extern "C" {
#endif
/**************************************
* Includes
**************************************/
#include <stddef.h> /* size_t */
/**************************************
* Version
**************************************/
#define ZSTD_VERSION_MAJOR 0 /* for breaking interface changes */
#define ZSTD_VERSION_MINOR 0 /* for new (non-breaking) interface capabilities */
#define ZSTD_VERSION_RELEASE 2 /* for tweaks, bug-fixes, or development */
#define ZSTD_VERSION_NUMBER (ZSTD_VERSION_MAJOR *100*100 + ZSTD_VERSION_MINOR *100 + ZSTD_VERSION_RELEASE)
unsigned ZSTD_versionNumber (void);
/**************************************
* Simple one-step functions
**************************************/
size_t ZSTD_compress( void* dst, size_t maxDstSize,
const void* src, size_t srcSize);
size_t ZSTD_decompress( void* dst, size_t maxOriginalSize,
const void* src, size_t compressedSize);
/*
ZSTD_compress() :
Compresses 'srcSize' bytes from buffer 'src' into buffer 'dst', of maximum size 'dstSize'.
Destination buffer should be sized to handle worst cases situations (input data not compressible).
Worst case size evaluation is provided by function ZSTD_compressBound().
return : the number of bytes written into buffer 'dst'
or an error code if it fails (which can be tested using ZSTD_isError())
ZSTD_decompress() :
compressedSize : is obviously the source size
maxOriginalSize : is the size of the 'dst' buffer, which must be already allocated.
It must be equal or larger than originalSize, otherwise decompression will fail.
return : the number of bytes decompressed into destination buffer (originalSize)
or an errorCode if it fails (which can be tested using ZSTD_isError())
*/
/**************************************
* Tool functions
**************************************/
size_t ZSTD_compressBound(size_t srcSize); /* maximum compressed size */
/* Error Management */
unsigned ZSTD_isError(size_t code); /* tells if a return value is an error code */
const char* ZSTD_getErrorName(size_t code); /* provides error code string (useful for debugging) */
#if defined (__cplusplus)
}
#endif

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