This commit is contained in:
Evgeniy Gatov 2015-04-21 17:21:52 +03:00
commit 4ebd2f703e
141 changed files with 1948 additions and 866 deletions

View File

@ -8,3 +8,8 @@ else()
add_custom_target(check COMMAND ${CMAKE_CTEST_COMMAND}
--force-new-ctest-process --output-on-failure)
endif()
macro (add_check target)
add_test(test_${target} ${target})
add_dependencies(check ${target})
endmacro (add_check)

View File

@ -21,7 +21,7 @@ SELECT UserID FROM {table} WHERE UserID = 12345678901234567890;
SELECT count() FROM {table} WHERE URL LIKE '%metrika%';
SELECT SearchPhrase, any(URL), count() AS c FROM {table} WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;
SELECT SearchPhrase, any(URL), any(Title), count() AS c, uniq(UserID) FROM {table} WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;
SELECT * FROM {table} WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;
SELECT * FROM {table} PREWHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;
SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;
SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;
SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;

View File

@ -3,10 +3,13 @@
#include <boost/program_options.hpp>
#include <boost/algorithm/string.hpp>
#include <DB/DataStreams/AsynchronousBlockInputStream.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/Interpreters/Context.h>
#include <DB/IO/copyData.h>
#include <DB/IO/ReadBufferFromIStream.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/Storages/StorageMemory.h>
#include <DB/Client/Connection.h>
#include <Poco/Net/HTMLForm.h>
#include <Poco/Net/PartHandler.h>
#include <Poco/Net/MessageHeader.h>

View File

@ -14,6 +14,13 @@
M(FileOpen) \
M(Seek) \
M(ReadBufferFromFileDescriptorRead) \
M(ReadBufferFromFileDescriptorReadBytes) \
M(WriteBufferFromFileDescriptorWrite) \
M(WriteBufferFromFileDescriptorWriteBytes) \
M(ReadBufferAIORead) \
M(ReadBufferAIOReadBytes) \
M(WriteBufferAIOWrite) \
M(WriteBufferAIOWriteBytes) \
M(ReadCompressedBytes) \
M(CompressedReadBufferBlocks) \
M(CompressedReadBufferBytes) \
@ -27,6 +34,10 @@
M(FunctionExecute) \
M(MarkCacheHits) \
M(MarkCacheMisses) \
M(CreatedReadBufferOrdinary) \
M(CreatedReadBufferAIO) \
M(CreatedWriteBufferOrdinary) \
M(CreatedWriteBufferAIO) \
\
M(ReplicatedPartFetches) \
M(ReplicatedPartFailedFetches) \

View File

@ -55,7 +55,10 @@ protected:
if (res)
break;
else
{
(*current_stream)->readSuffix();
++current_stream;
}
}
return res;

View File

@ -49,6 +49,9 @@ public:
return res.str();
}
/// Берёт totals только из основного источника, а не из источников подзапросов.
const Block & getTotals() override;
protected:
Block readImpl() override;

View File

@ -14,7 +14,7 @@ using Poco::SharedPtr;
/** Выполняет над блоком вычисление некоторого выражения.
* Выражение состоит из идентификаторов столбцов из блока, констант, обычных функций.
* Например: hits * 2 + 3, instr("yandex", url)
* Например: hits * 2 + 3, url LIKE '%yandex%'
* Выражение не меняет количество строк в потоке, и обрабатывает каждую строку независимо от других.
*/
class ExpressionBlockInputStream : public IProfilingBlockInputStream
@ -40,9 +40,7 @@ public:
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back()))
{
totals = child->getTotals();
if (totals)
expression->execute(totals);
expression->executeOnTotals(totals);
}
return totals;

View File

@ -28,6 +28,8 @@ public:
return res.str();
}
static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows);
protected:
Block readImpl() override;

View File

@ -21,6 +21,8 @@ public:
void write(const Block & block) override;
void flush() override { ostr.next(); }
static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit);
private:
WriteBuffer & ostr;
UInt64 client_revision;

View File

@ -70,6 +70,10 @@ protected:
if (!executed)
{
executed = true;
Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); };
aggregator.setCancellationHook(hook);
AggregatedDataVariantsPtr data_variants = executeAndMerge();
if (data_variants)

View File

@ -84,7 +84,7 @@ public:
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
if (hasNoQueryInProgress() || hasThrownException())
if (!isQueryPending() || hasThrownException())
return;
tryCancel("Cancelling query");
@ -97,7 +97,7 @@ public:
* все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы
* эти соединения не остались висеть в рассихронизированном состоянии.
*/
if (established || isQueryInProgress())
if (established || isQueryPending())
parallel_replicas->disconnect();
}
@ -212,7 +212,7 @@ protected:
* - получили с одной реплики неизвестный пакет;
* - то больше читать ничего не нужно.
*/
if (hasNoQueryInProgress() || hasThrownException())
if (!isQueryPending() || hasThrownException())
return;
/** Если ещё прочитали не все данные, но они больше не нужны.
@ -251,16 +251,10 @@ protected:
parallel_replicas = std::make_unique<ParallelReplicas>(pool, parallel_replicas_settings, throttler);
}
/// Возвращает true, если запрос отправлен, а ещё не выполнен.
bool isQueryInProgress() const
/// Возвращает true, если запрос отправлен.
bool isQueryPending() const
{
return sent_query && !finished && !was_cancelled;
}
/// Возвращает true, если никакой запрос не отправлен или один запрос уже выполнен.
bool hasNoQueryInProgress() const
{
return !sent_query || finished;
return sent_query && !finished;
}
/// Возвращает true, если исключение было выкинуто.

View File

@ -3,6 +3,8 @@
#include <DB/DataTypes/IDataType.h>
#include <DB/Columns/ColumnTuple.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/DataStreams/NativeBlockInputStream.h>
#include <DB/DataStreams/NativeBlockOutputStream.h>
namespace DB
@ -115,14 +117,7 @@ public:
{
const ColumnTuple & real_column = static_cast<const ColumnTuple &>(column);
for (size_t i = 0, size = elems.size(); i < size; ++i)
{
const IColumn & nested_column = *real_column.getData().getByPosition(i).column;
if (nested_column.isConst())
elems[i]->serializeBinary(*static_cast<const IColumnConst &>(nested_column).convertToFullColumn(), ostr, offset, limit);
else
elems[i]->serializeBinary(nested_column, ostr, offset, limit);
}
NativeBlockOutputStream::writeData(*elems[i], real_column.getData().getByPosition(i).column, ostr, offset, limit);
}
/** limit обязательно должен быть в точности равен количеству сериализованных значений.
@ -133,7 +128,7 @@ public:
{
ColumnTuple & real_column = static_cast<ColumnTuple &>(column);
for (size_t i = 0, size = elems.size(); i < size; ++i)
elems[i]->deserializeBinary(*real_column.getData().getByPosition(i).column, istr, limit, avg_value_size_hint);
NativeBlockInputStream::readData(*elems[i], *real_column.getData().getByPosition(i).column, istr, limit);
}
ColumnPtr createColumn() const

View File

@ -4,6 +4,8 @@
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/OwningBufferBlockInputStream.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/DataStreams/FormatFactory.h>
#include <Poco/Timestamp.h>
#include <Poco/File.h>

View File

@ -4,6 +4,7 @@
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Common/Arena.h>
#include <statdaemons/ext/range.hpp>
#include <vector>
#include <tuple>

View File

@ -9,6 +9,7 @@
#include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp>
#include <mysqlxx/Query.h>
#include <mysqlxx/PoolWithFailover.h>
#include <vector>
#include <string>

View File

@ -11,13 +11,16 @@
#include <DB/Columns/ColumnString.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/Dictionaries.h>
#include <DB/Interpreters/ExternalDictionaries.h>
#include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp>
#include <DB/Dictionaries/FlatDictionary.h>
#include <DB/Dictionaries/HashedDictionary.h>
#include <DB/Dictionaries/CacheDictionary.h>
#include <statdaemons/ext/range.hpp>
namespace DB
{

View File

@ -20,6 +20,8 @@
#include <DB/Common/HashTable/Hash.h>
#include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp>
#include <stats/IntHash.h>
@ -603,6 +605,215 @@ public:
};
struct URLHashImpl
{
static UInt64 apply(const char * data, const std::size_t size)
{
/// do not take last slash, '?' or '#' character into account
if (size > 0 && (data[size - 1] == '/' || data[size - 1] == '?' || data[size - 1] == '#'))
return CityHash64(data, size - 1);
return CityHash64(data, size);
}
};
struct URLHierarchyHashImpl
{
static std::size_t findLevelLength(const UInt64 level, const char * begin, const char * const end)
{
auto pos = begin;
/// Распарсим всё, что идёт до пути
/// Предположим, что протокол уже переведён в нижний регистр.
while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9')))
++pos;
/** Будем вычислять иерархию только для URL-ов, в которых есть протокол, и после него идут два слеша.
* (http, file - подходят, mailto, magnet - не подходят), и после двух слешей ещё хоть что-нибудь есть
* Для остальных просто вернём полный URL как единственный элемент иерархии.
*/
if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end))
{
pos = end;
return 0 == level ? pos - begin : 0;
}
/// Доменом для простоты будем считать всё, что после протокола и двух слешей, до следующего слеша или до ? или до #
while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos != end)
++pos;
if (0 == level)
return pos - begin;
UInt64 current_level = 0;
while (current_level != level && pos < end)
{
/// Идём до следующего / или ? или #, пропуская все те, что вначале.
while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos == end)
break;
while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos != end)
++pos;
++current_level;
}
return current_level == level ? pos - begin : 0;
}
static UInt64 apply(const UInt64 level, const char * data, const std::size_t size)
{
return URLHashImpl::apply(data, findLevelLength(level, data, data + size));
}
};
class FunctionURLHash : public IFunction
{
public:
static constexpr auto name = "URLHash";
static IFunction * create(const Context &) { return new FunctionURLHash; }
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
const auto arg_count = arguments.size();
if (arg_count != 1 && arg_count != 2)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(arg_count) + ", should be 1 or 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto first_arg = arguments.front().get();
if (!typeid_cast<const DataTypeString *>(first_arg))
throw Exception{
"Illegal type " + first_arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
if (arg_count == 2)
{
const auto second_arg = arguments.back().get();
if (!typeid_cast<const DataTypeUInt8 *>(second_arg) &&
!typeid_cast<const DataTypeUInt16 *>(second_arg) &&
!typeid_cast<const DataTypeUInt32 *>(second_arg) &&
!typeid_cast<const DataTypeUInt64 *>(second_arg) &&
!typeid_cast<const DataTypeInt8 *>(second_arg) &&
!typeid_cast<const DataTypeInt16 *>(second_arg) &&
!typeid_cast<const DataTypeInt32 *>(second_arg) &&
!typeid_cast<const DataTypeInt64 *>(second_arg))
throw Exception{
"Illegal type " + second_arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
return new DataTypeUInt64;
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
{
const auto arg_count = arguments.size();
if (arg_count == 1)
executeSingleArg(block, arguments, result);
else if (arg_count == 2)
executeTwoArgs(block, arguments, result);
else
throw std::logic_error{"got into IFunction::execute with unexpected number of arguments"};
}
private:
void executeSingleArg(Block & block, const ColumnNumbers & arguments, const std::size_t result) const
{
const auto col_untyped = block.getByPosition(arguments.front()).column.get();
if (const auto col_from = typeid_cast<const ColumnString *>(col_untyped))
{
const auto size = col_from->size();
const auto col_to = new ColumnVector<UInt64>{size};
block.getByPosition(result).column = col_to;
const auto & chars = col_from->getChars();
const auto & offsets = col_from->getOffsets();
auto & out = col_to->getData();
for (const auto i : ext::range(0, size))
out[i] = URLHashImpl::apply(
reinterpret_cast<const char *>(&chars[i == 0 ? 0 : offsets[i - 1]]),
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
}
else if (const auto col_from = typeid_cast<const ColumnConstString *>(col_untyped))
{
block.getByPosition(result).column = new ColumnConstUInt64{
col_from->size(),
URLHashImpl::apply(col_from->getData().data(), col_from->getData().size())
};
}
else
throw Exception{
"Illegal column " + block.getByPosition(arguments[0]).column->getName() +
" of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
}
void executeTwoArgs(Block & block, const ColumnNumbers & arguments, const std::size_t result) const
{
const auto level_col = block.getByPosition(arguments.back()).column.get();
if (!level_col->isConst())
throw Exception{
"Second argument of function " + getName() + " must be an integral constant",
ErrorCodes::ILLEGAL_COLUMN
};
const auto level = level_col->get64(0);
const auto col_untyped = block.getByPosition(arguments.front()).column.get();
if (const auto col_from = typeid_cast<const ColumnString *>(col_untyped))
{
const auto size = col_from->size();
const auto col_to = new ColumnVector<UInt64>{size};
block.getByPosition(result).column = col_to;
const auto & chars = col_from->getChars();
const auto & offsets = col_from->getOffsets();
auto & out = col_to->getData();
for (const auto i : ext::range(0, size))
out[i] = URLHierarchyHashImpl::apply(level,
reinterpret_cast<const char *>(&chars[i == 0 ? 0 : offsets[i - 1]]),
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
}
else if (const auto col_from = typeid_cast<const ColumnConstString *>(col_untyped))
{
block.getByPosition(result).column = new ColumnConstUInt64{
col_from->size(),
URLHierarchyHashImpl::apply(level, col_from->getData().data(), col_from->getData().size())
};
}
else
throw Exception{
"Illegal column " + block.getByPosition(arguments[0]).column->getName() +
" of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
}
};
struct NameHalfMD5 { static constexpr auto name = "halfMD5"; };
struct NameSipHash64 { static constexpr auto name = "sipHash64"; };
struct NameCityHash64 { static constexpr auto name = "cityHash64"; };

View File

@ -20,6 +20,7 @@ private:
const std::string path;
UncompressedCache * cache;
size_t buf_size;
size_t estimated_size;
size_t aio_threshold;
/// SharedPtr - для ленивой инициализации (только в случае кэш-промаха).
@ -33,7 +34,7 @@ private:
{
if (!file_in)
{
file_in = createReadBufferFromFileBase(path, aio_threshold, buf_size);
file_in = createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size);
compressed_in = &*file_in;
}
}
@ -81,9 +82,10 @@ private:
}
public:
CachedCompressedReadBuffer(const std::string & path_, UncompressedCache * cache_, size_t aio_threshold_,
size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE)
: ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_), aio_threshold(aio_threshold_), file_pos(0)
CachedCompressedReadBuffer(const std::string & path_, UncompressedCache * cache_, size_t estimated_size_,
size_t aio_threshold_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE)
: ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_),
estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0)
{
}

View File

@ -42,9 +42,9 @@ private:
}
public:
CompressedReadBufferFromFile(const std::string & path, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE)
CompressedReadBufferFromFile(const std::string & path, size_t estimated_size, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE)
: BufferWithOwnMemory<ReadBuffer>(0),
p_file_in(createReadBufferFromFileBase(path, aio_threshold, buf_size)),
p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size)),
file_in(*p_file_in)
{
compressed_in = &file_in;

View File

@ -145,7 +145,7 @@ public:
}
protected:
/// Количество игнорируемых байтов с начальной позиции буфера working_buffer.
/// Количество игнорируемых байт с начальной позиции буфера working_buffer.
size_t working_buffer_offset = 0;
private:

View File

@ -10,7 +10,6 @@
#include <limits>
#include <unistd.h>
#include <fcntl.h>
#include <sys/uio.h>
namespace DB
{
@ -29,8 +28,8 @@ public:
void setMaxBytes(size_t max_bytes_read_);
off_t getPositionInFile() override { return first_unread_pos_in_file - (working_buffer.end() - pos); }
std::string getFileName() const noexcept override { return filename; }
int getFD() const noexcept override { return fd; }
std::string getFileName() const override { return filename; }
int getFD() const override { return fd; }
private:
///
@ -68,7 +67,7 @@ private:
size_t max_bytes_read = std::numeric_limits<size_t>::max();
/// Количество запрашиваемых байтов.
size_t requested_byte_count = 0;
/// Количество прочитанных байтов при последнем запросе.
/// Количество прочитанных байт при последнем запросе.
ssize_t bytes_read = 0;
/// Итоговое количество прочитанных байтов.
size_t total_bytes_read = 0;

View File

@ -16,8 +16,8 @@ public:
virtual ~ReadBufferFromFileBase();
off_t seek(off_t off, int whence = SEEK_SET);
virtual off_t getPositionInFile() = 0;
virtual std::string getFileName() const noexcept = 0;
virtual int getFD() const noexcept = 0;
virtual std::string getFileName() const = 0;
virtual int getFD() const = 0;
protected:
virtual off_t doSeek(off_t off, int whence) = 0;

View File

@ -24,7 +24,7 @@ class ReadBufferFromFileDescriptor : public ReadBufferFromFileBase
protected:
int fd;
off_t pos_in_file; /// Какому сдвигу в файле соответствует working_buffer.end().
bool nextImpl()
{
size_t bytes_read = 0;
@ -35,7 +35,7 @@ protected:
ssize_t res = ::read(fd, internal_buffer.begin(), internal_buffer.size());
if (!res)
break;
if (-1 == res && errno != EINTR)
throwFromErrno("Cannot read from file " + getFileName(), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
@ -46,7 +46,10 @@ protected:
pos_in_file += bytes_read;
if (bytes_read)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read);
working_buffer.resize(bytes_read);
}
else
return false;
@ -54,7 +57,7 @@ protected:
}
/// Имя или описание файла
virtual std::string getFileName() const noexcept override
virtual std::string getFileName() const override
{
return "(fd = " + toString(fd) + ")";
}
@ -63,7 +66,7 @@ public:
ReadBufferFromFileDescriptor(int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0)
: ReadBufferFromFileBase(buf_size, existing_memory, alignment), fd(fd_), pos_in_file(0) {}
int getFD() const noexcept override
int getFD() const override
{
return fd;
}

View File

@ -70,6 +70,6 @@ public:
}
};
typedef Poco::SharedPtr<UncompressedCache> UncompressedCachePtr;
typedef std::shared_ptr<UncompressedCache> UncompressedCachePtr;
}

View File

@ -28,8 +28,8 @@ public:
off_t getPositionInFile() override;
void truncate(off_t length = 0) override;
void sync() override;
std::string getFileName() const noexcept override { return filename; }
int getFD() const noexcept override { return fd; }
std::string getFileName() const override { return filename; }
int getFD() const override { return fd; }
private:
///
@ -60,7 +60,7 @@ private:
/// Количество байтов, которые будут записаны на диск.
off_t bytes_to_write = 0;
/// Количество записанных байтов при последнем запросе.
/// Количество записанных байт при последнем запросе.
off_t bytes_written = 0;
/// Количество нулевых байтов, которые надо отрезать c конца файла
/// после завершения операции записи данных.

View File

@ -18,8 +18,8 @@ public:
virtual off_t getPositionInFile() = 0;
virtual void truncate(off_t length) = 0;
virtual void sync() = 0;
virtual std::string getFileName() const noexcept = 0;
virtual int getFD() const noexcept = 0;
virtual std::string getFileName() const = 0;
virtual int getFD() const = 0;
protected:
virtual off_t doSeek(off_t off, int whence) = 0;

View File

@ -5,6 +5,7 @@
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/IO/WriteBufferFromFileBase.h>
#include <DB/IO/WriteBuffer.h>
@ -21,7 +22,7 @@ class WriteBufferFromFileDescriptor : public WriteBufferFromFileBase
{
protected:
int fd;
void nextImpl()
{
if (!offset())
@ -30,6 +31,8 @@ protected:
size_t bytes_written = 0;
while (bytes_written != offset())
{
ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWrite);
ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written);
if ((-1 == res || 0 == res) && errno != EINTR)
@ -38,10 +41,12 @@ protected:
if (res > 0)
bytes_written += res;
}
ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteBytes, bytes_written);
}
/// Имя или описание файла
virtual std::string getFileName() const noexcept override
virtual std::string getFileName() const override
{
return "(fd = " + toString(fd) + ")";
}
@ -70,7 +75,7 @@ public:
}
}
int getFD() const noexcept override
int getFD() const override
{
return fd;
}

View File

@ -2,12 +2,19 @@
#include <DB/IO/ReadBufferFromFileBase.h>
#include <string>
#include <sys/stat.h>
namespace DB
{
/** Создать объект для чтения данных из файла.
* estimated_size - количество байтов, которые надо читать
* aio_threshold - минимальное количество байт для асинхронных операций чтения
*
* Если aio_threshold = 0 или estimated_size < aio_threshold, операции чтения выполняются синхронно.
* В противном случае операции чтения выполняются асинхронно.
*/
ReadBufferFromFileBase * createReadBufferFromFileBase(const std::string & filename_,
size_t estimated_size,
size_t aio_threshold,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
int flags_ = -1,

View File

@ -2,11 +2,17 @@
#include <DB/IO/WriteBufferFromFileBase.h>
#include <string>
#include <sys/stat.h>
namespace DB
{
/** Создать объект для записи данных в файл.
* estimated_size - количество байтов, которые надо записать
* aio_threshold - минимальное количество байт для асинхронных операций записи
*
* Если aio_threshold = 0 или estimated_size < aio_threshold, операции записи выполняются синхронно.
* В противном случае операции записи выполняются асинхронно.
*/
WriteBufferFromFileBase * createWriteBufferFromFileBase(const std::string & filename_,
size_t estimated_size,
size_t aio_threshold,

View File

@ -2,6 +2,7 @@
#include <mutex>
#include <memory>
#include <functional>
#include <Yandex/logger_useful.h>
#include <statdaemons/threadpool.hpp>
@ -523,6 +524,7 @@ struct AggregatedDataVariants : private boost::noncopyable
AggregatedDataVariants() : aggregates_pools(1, new Arena), aggregates_pool(&*aggregates_pools.back()) {}
bool empty() const { return type == Type::EMPTY; }
void invalidate() { type = Type::EMPTY; }
~AggregatedDataVariants();
@ -682,7 +684,8 @@ public:
: key_names(key_names_), 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_)
compiler(compiler_), min_count_to_compile(min_count_to_compile_), group_by_two_level_threshold(group_by_two_level_threshold_),
isCancelled([]() { return false; })
{
std::sort(key_names.begin(), key_names.end());
key_names.erase(std::unique(key_names.begin(), key_names.end()), key_names.end());
@ -723,6 +726,12 @@ public:
*/
void mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads);
using CancellationHook = std::function<bool()>;
/** Установить функцию, которая проверяет, можно ли прервать текущую задачу.
*/
void setCancellationHook(const CancellationHook cancellation_hook);
/// Для IBlockInputStream.
String getID() const;
@ -786,6 +795,9 @@ protected:
*/
size_t group_by_two_level_threshold;
/// Возвращает true, если можно прервать текущую задачу.
CancellationHook isCancelled;
/** Если заданы только имена столбцов (key_names, а также aggregates[i].column_name), то вычислить номера столбцов.
* Сформировать блок - пример результата.
*/

View File

@ -1,44 +1,45 @@
#pragma once
#include <map>
#include <set>
#include <Poco/SharedPtr.h>
#include <Poco/Mutex.h>
#include <Yandex/logger_useful.h>
#include <functional>
#include <memory>
#include <DB/Core/Types.h>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Common/Macros.h>
#include <DB/IO/UncompressedCache.h>
#include <DB/Storages/MarkCache.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/Storages/IStorage.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
#include <DB/Storages/MergeTree/MergeList.h>
#include <DB/TableFunctions/TableFunctionFactory.h>
#include <DB/Interpreters/Settings.h>
#include <DB/Interpreters/Users.h>
#include <DB/Interpreters/Quota.h>
#include <DB/Interpreters/Dictionaries.h>
#include <DB/Interpreters/ExternalDictionaries.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Interpreters/Cluster.h>
#include <DB/Interpreters/InterserverIOHandler.h>
#include <DB/Interpreters/Compiler.h>
#include <DB/Client/ConnectionPool.h>
#include <statdaemons/ConfigProcessor.h>
#include <zkutil/ZooKeeper.h>
#include <DB/Storages/IStorage.h>
#include <DB/IO/CompressedStream.h>
#include <Poco/Net/IPAddress.h>
namespace zkutil
{
class ZooKeeper;
}
namespace DB
{
class ContextShared;
class QuotaForIntervals;
class TableFunctionFactory;
class AggregateFunctionFactory;
class FormatFactory;
class Dictionaries;
class ExternalDictionaries;
class InterserverIOHandler;
class BackgroundProcessingPool;
class MergeList;
class Cluster;
class Compiler;
class MarkCache;
class UncompressedCache;
class ProcessList;
class ProcessListElement;
class Macros;
class Progress;
using Poco::SharedPtr;
/// имя таблицы -> таблица
typedef std::map<String, StoragePtr> Tables;
@ -53,112 +54,6 @@ typedef std::pair<String, String> DatabaseAndTableName;
typedef std::map<DatabaseAndTableName, std::set<DatabaseAndTableName>> ViewDependencies;
typedef std::vector<DatabaseAndTableName> Dependencies;
/** Набор известных объектов, которые могут быть использованы в запросе.
* Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен.
*/
struct ContextShared
{
Logger * log = &Logger::get("Context"); /// Логгер.
struct AfterDestroy
{
Logger * log;
AfterDestroy(Logger * log_) : log(log_) {}
~AfterDestroy()
{
#ifndef DBMS_CLIENT
LOG_INFO(log, "Uninitialized shared context.");
#endif
}
} after_destroy {log};
mutable Poco::Mutex mutex; /// Для доступа и модификации разделяемых объектов.
mutable zkutil::ZooKeeperPtr zookeeper; /// Клиент для ZooKeeper.
String interserver_io_host; /// Имя хоста по которым это сервер доступен для других серверов.
int interserver_io_port; /// и порт,
String path; /// Путь к директории с данными, со слешем на конце.
String tmp_path; /// Путь ко временным файлам, возникающим при обработке запроса.
Databases databases; /// Список БД и таблиц в них.
TableFunctionFactory table_function_factory; /// Табличные функции.
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
DataTypeFactory data_type_factory; /// Типы данных.
FormatFactory format_factory; /// Форматы.
mutable SharedPtr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво.
mutable SharedPtr<ExternalDictionaries> external_dictionaries;
Users users; /// Известные пользователи.
Quotas quotas; /// Известные квоты на использование ресурсов.
mutable UncompressedCachePtr uncompressed_cache; /// Кэш разжатых блоков.
mutable MarkCachePtr mark_cache; /// Кэш засечек в сжатых файлах.
ProcessList process_list; /// Исполняющиеся в данный момент запросы.
MergeList merge_list; /// Список выполняемых мерджей (для (Replicated)?MergeTree)
ViewDependencies view_dependencies; /// Текущие зависимости
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
InterserverIOHandler interserver_io_handler; /// Обработчик для межсерверной передачи данных.
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
Macros macros; /// Подстановки из конфига.
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
/// Кластеры для distributed таблиц
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
Poco::SharedPtr<Clusters> clusters;
bool shutdown_called = false;
~ContextShared()
{
#ifndef DBMS_CLIENT
LOG_INFO(log, "Uninitializing shared context.");
#endif
try
{
shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
/** Выполнить сложную работу по уничтожению объектов заранее.
*/
void shutdown()
{
if (shutdown_called)
return;
shutdown_called = true;
/** В этот момент, некоторые таблицы могут иметь потоки,
* которые модифицируют список таблиц, и блокируют наш mutex (см. StorageChunkMerger).
* Чтобы корректно их завершить, скопируем текущий список таблиц,
* и попросим их всех закончить свою работу.
* Потом удалим все объекты с таблицами.
*/
Databases current_databases;
{
Poco::ScopedLock<Poco::Mutex> lock(mutex);
current_databases = databases;
}
for (Databases::iterator it = current_databases.begin(); it != current_databases.end(); ++it)
for (Tables::iterator jt = it->second.begin(); jt != it->second.end(); ++jt)
jt->second->shutdown();
{
Poco::ScopedLock<Poco::Mutex> lock(mutex);
databases.clear();
}
}
};
/** Набор известных объектов, которые могут быть использованы в запросе.
* Состоит из разделяемой части (всегда общей для всех сессий и запросов)
@ -169,31 +64,37 @@ struct ContextShared
class Context
{
private:
typedef SharedPtr<ContextShared> Shared;
Shared shared = new ContextShared;
typedef std::shared_ptr<ContextShared> Shared;
Shared shared;
String user; /// Текущий пользователь.
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
QuotaForIntervalsPtr quota = new QuotaForIntervals; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
String current_database; /// Текущая БД.
String current_query_id; /// Id текущего запроса.
NamesAndTypesList columns; /// Столбцы текущей обрабатываемой таблицы.
Settings settings; /// Настройки выполнения запроса.
using ProgressCallback = std::function<void(const Progress & progress)>;
ProgressCallback progress_callback; /// Колбек для отслеживания прогресса выполнения запроса.
ProcessList::Element * process_list_elem = nullptr; /// Для отслеживания общего количества потраченных на запрос ресурсов.
ProcessListElement * process_list_elem = nullptr; /// Для отслеживания общего количества потраченных на запрос ресурсов.
String default_format; /// Формат, используемый, если сервер сам форматирует данные, и если в запросе не задан FORMAT.
/// То есть, используется в HTTP-интерфейсе. Может быть не задан - тогда используется некоторый глобальный формат по-умолчанию.
String default_format; /// Формат, используемый, если сервер сам форматирует данные, и если в запросе не задан FORMAT.
/// То есть, используется в HTTP-интерфейсе. Может быть не задан - тогда используется некоторый глобальный формат по-умолчанию.
Tables external_tables; /// Временные таблицы.
Context * session_context = nullptr; /// Контекст сессии или nullptr, если его нет. (Возможно, равен this.)
Context * global_context = nullptr; /// Глобальный контекст или nullptr, если его нет. (Возможно, равен this.)
public:
Context();
~Context();
String getPath() const;
String getTemporaryPath() const;
void setPath(const String & path);
void setTemporaryPath(const String & path);
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
/** Забрать список пользователей, квот и профилей настроек из этого конфига.
* Список пользователей полностью заменяется.
* Накопленные значения у квоты не сбрасываются, если квота не удалена.
@ -256,16 +157,16 @@ public:
/// Установить настройку по имени. Прочитать значение в текстовом виде из строки (например, из конфига, или из параметра URL).
void setSetting(const String & name, const std::string & value);
const TableFunctionFactory & getTableFunctionFactory() const { return shared->table_function_factory; }
const AggregateFunctionFactory & getAggregateFunctionFactory() const { return shared->aggregate_function_factory; }
const DataTypeFactory & getDataTypeFactory() const { return shared->data_type_factory; }
const FormatFactory & getFormatFactory() const { return shared->format_factory; }
const TableFunctionFactory & getTableFunctionFactory() const;
const AggregateFunctionFactory & getAggregateFunctionFactory() const;
const DataTypeFactory & getDataTypeFactory() const;
const FormatFactory & getFormatFactory() const;
const Dictionaries & getDictionaries() const;
const ExternalDictionaries & getExternalDictionaries() const;
void tryCreateDictionaries() const;
void tryCreateExternalDictionaries() const;
InterserverIOHandler & getInterserverIOHandler() { return shared->interserver_io_handler; }
InterserverIOHandler & getInterserverIOHandler();
/// Как другие серверы могут обратиться к этому для скачивания реплицируемых данных.
void setInterserverIOAddress(const String & host, UInt16 port);
@ -275,11 +176,11 @@ public:
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;
/// Для методов ниже может быть необходимо захватывать mutex самостоятельно.
Poco::Mutex & getMutex() const { return shared->mutex; }
Poco::Mutex & getMutex() const;
/// Метод getDatabases не потокобезопасен. При работе со списком БД и таблиц, вы должны захватить mutex.
const Databases & getDatabases() const { return shared->databases; }
Databases & getDatabases() { return shared->databases; }
const Databases & getDatabases() const;
Databases & getDatabases();
/// При работе со списком столбцов, используйте локальный контекст, чтобы никто больше его не менял.
const NamesAndTypesList & getColumns() const { return columns; }
@ -302,28 +203,28 @@ public:
/** Устанавливается в executeQuery и InterpreterSelectQuery. Затем используется в IProfilingBlockInputStream,
* чтобы обновлять и контролировать информацию об общем количестве потраченных на запрос ресурсов.
*/
void setProcessListElement(ProcessList::Element * elem);
void setProcessListElement(ProcessListElement * elem);
/// Может вернуть nullptr, если запрос не был вставлен в ProcessList.
ProcessList::Element * getProcessListElement();
ProcessListElement * getProcessListElement();
/// Список всех запросов.
ProcessList & getProcessList() { return shared->process_list; }
const ProcessList & getProcessList() const { return shared->process_list; }
ProcessList & getProcessList();
const ProcessList & getProcessList() const;
MergeList & getMergeList() { return shared->merge_list; }
const MergeList & getMergeList() const { return shared->merge_list; }
MergeList & getMergeList();
const MergeList & getMergeList() const;
/// Создать кэш разжатых блоков указанного размера. Это можно сделать только один раз.
void setUncompressedCache(size_t max_size_in_bytes);
UncompressedCachePtr getUncompressedCache() const;
std::shared_ptr<UncompressedCache> getUncompressedCache() const;
void setZooKeeper(zkutil::ZooKeeperPtr zookeeper);
void setZooKeeper(std::shared_ptr<zkutil::ZooKeeper> zookeeper);
/// Если в момент вызова текущая сессия просрочена, синхронно создает и возвращает новую вызовом startNewSession().
zkutil::ZooKeeperPtr getZooKeeper() const;
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
/// Создать кэш засечек указанного размера. Это можно сделать только один раз.
void setMarkCache(size_t cache_size_in_bytes);
MarkCachePtr getMarkCache() const;
std::shared_ptr<MarkCache> getMarkCache() const;
BackgroundProcessingPool & getBackgroundPool();
@ -340,7 +241,10 @@ public:
Compiler & getCompiler();
void shutdown() { shared->shutdown(); }
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;
void shutdown();
private:
const Dictionaries & getDictionariesImpl(bool throw_on_error) const;

View File

@ -1,5 +1,6 @@
#pragma once
#include <thread>
#include <Yandex/MultiVersion.h>
#include <Yandex/logger_useful.h>
#include <statdaemons/RegionsHierarchies.h>

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/DataTypes/IDataType.h>
#include <DB/DataStreams/IBlockInputStream.h>
#include <DB/Interpreters/Settings.h>
#include <DB/Core/Names.h>
#include <DB/Core/ColumnWithNameAndType.h>
@ -67,7 +68,7 @@ public:
NameSet array_joined_columns;
/// Для JOIN
Join * join = nullptr;
const Join * join = nullptr;
NamesAndTypesList columns_added_by_join;
/// Для PROJECT.
@ -131,7 +132,7 @@ public:
return a;
}
static ExpressionAction ordinaryJoin(Join * join_, const NamesAndTypesList & columns_added_by_join_)
static ExpressionAction ordinaryJoin(const Join * join_, const NamesAndTypesList & columns_added_by_join_)
{
ExpressionAction a;
a.type = JOIN;
@ -152,6 +153,7 @@ private:
std::vector<ExpressionAction> getPrerequisites(Block & sample_block);
void prepare(Block & sample_block);
void execute(Block & block) const;
void executeOnTotals(Block & block) const;
};
@ -165,20 +167,18 @@ public:
ExpressionActions(const NamesAndTypesList & input_columns_, const Settings & settings_)
: input_columns(input_columns_), settings(settings_)
{
for (NamesAndTypesList::iterator it = input_columns.begin(); it != input_columns.end(); ++it)
{
sample_block.insert(ColumnWithNameAndType(nullptr, it->type, it->name));
}
for (const auto & input_elem : input_columns)
sample_block.insert(ColumnWithNameAndType(nullptr, input_elem.type, input_elem.name));
}
/// Для константных столбцов в input_columns_ могут содержаться сами столбцы.
ExpressionActions(const ColumnsWithNameAndType & input_columns_, const Settings & settings_)
: settings(settings_)
: settings(settings_)
{
for (ColumnsWithNameAndType::const_iterator it = input_columns_.begin(); it != input_columns_.end(); ++it)
for (const auto & input_elem : input_columns_)
{
input_columns.emplace_back(it->name, it->type);
sample_block.insert(*it);
input_columns.emplace_back(input_elem.name, input_elem.type);
sample_block.insert(input_elem);
}
}
@ -227,6 +227,11 @@ public:
/// Выполнить выражение над блоком. Блок должен содержать все столбцы , возвращаемые getRequiredColumns.
void execute(Block & block) const;
/** Выполнить выражение над блоком тотальных значений.
* Почти не отличается от execute. Разница лишь при выполнении JOIN-а.
*/
void executeOnTotals(Block & block) const;
/// Получить блок-образец, содержащий имена и типы столбцов результата.
const Block & getSampleBlock() const { return sample_block; }
@ -236,6 +241,8 @@ public:
static std::string getSmallestColumn(const NamesAndTypesList & columns);
BlockInputStreamPtr createStreamWithNonJoinedDataIfFullOrRightJoin(size_t max_block_size) const;
private:
NamesAndTypesList input_columns;
Actions actions;

View File

@ -61,10 +61,10 @@ public:
}
/// Есть ли в выражении агрегатные функции или секция GROUP BY или HAVING.
bool hasAggregation() { return has_aggregation; }
bool hasAggregation() const { return has_aggregation; }
/// Получить список ключей агрегирования и описаний агрегатных функций, если в запросе есть GROUP BY.
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates);
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const;
/** Получить набор столбцов, которых достаточно прочитать из таблицы для вычисления выражения.
* Не учитываются столбцы, добавляемые из другой таблицы путём JOIN-а.
@ -97,7 +97,7 @@ public:
void appendSelect(ExpressionActionsChain & chain, bool only_types);
bool appendOrderBy(ExpressionActionsChain & chain, bool only_types);
/// Удаляет все столбцы кроме выбираемых SELECT, упорядочивает оставшиеся столбцы и переименовывает их в алиасы.
void appendProjectResult(ExpressionActionsChain & chain, bool only_types);
void appendProjectResult(ExpressionActionsChain & chain, bool only_types) const;
/// Если ast не запрос SELECT, просто получает все действия для вычисления выражения.
/// Если project_result, в выходном блоке останутся только вычисленные значения в нужном порядке, переименованные в алиасы.
@ -117,7 +117,7 @@ public:
/** Таблицы, которые надо будет отправить на удалённые серверы при распределённой обработке запроса.
*/
const Tables & getExternalTables() { return external_tables; }
const Tables & getExternalTables() const { return external_tables; }
/// Если ast - запрос SELECT, получает имена (алиасы) и типы столбцов из секции SELECT.
Block getSelectSampleBlock();
@ -238,9 +238,9 @@ private:
void getArrayJoinedColumns();
void getArrayJoinedColumnsImpl(ASTPtr ast);
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions);
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const;
void addJoinAction(ExpressionActionsPtr & actions, bool only_types);
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;
struct ScopeStack;
void getActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack);
@ -270,11 +270,10 @@ private:
StoragePtr getTable();
/// columns - столбцы, присутствующие до начала преобразований.
void initChain(ExpressionActionsChain & chain, NamesAndTypesList & columns);
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
void assertSelect();
void assertAggregation();
void assertArrayJoin();
void assertSelect() const;
void assertAggregation() const;
/** Создать Set из явного перечисления значений в запросе.
* Если create_ordered_set = true - создать структуру данных, подходящую для использования индекса.

View File

@ -50,9 +50,9 @@ private:
String from; /// Для FETCH PARTITION - путь в ZK к шарду, с которого скачивать партицию.
static PartitionCommand dropPartition(const Field & partition, bool detach)
static PartitionCommand dropPartition(const Field & partition, bool detach, bool unreplicated)
{
return {DROP_PARTITION, partition, detach};
return {DROP_PARTITION, partition, detach, unreplicated};
}
static PartitionCommand attachPartition(const Field & partition, bool unreplicated, bool part)

View File

@ -6,6 +6,7 @@
#include <DB/Interpreters/Context.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/DataStreams/BlockIO.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/DataStreams/copyData.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>

View File

@ -15,21 +15,21 @@ class InterpreterOptimizeQuery
{
public:
InterpreterOptimizeQuery(ASTPtr query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_),
aio_threshold(context_.getSettings().min_bytes_to_use_direct_io) {}
: query_ptr(query_ptr_), context(context_)
{
}
void execute()
{
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
StoragePtr table = context.getTable(ast.database, ast.table);
auto table_lock = table->lockStructure(true);
table->optimize(aio_threshold);
table->optimize(context.getSettings());
}
private:
ASTPtr query_ptr;
Context context;
size_t aio_threshold;
};

View File

@ -63,8 +63,12 @@ public:
*/
BlockInputStreamPtr execute();
/// Выполнить запрос без объединения потоков.
const BlockInputStreams & executeWithoutUnion();
/** Выполнить запрос без объединения потоков.
* Если force_no_union = false, выполнить объединения, которые не могут быть отложены до оформления окончательного результата.
* Если же force_no_union = true, не выполнять никаких объединений. Это позволяет оптимизировать запросы, в которых таблица
* является представлением.
*/
const BlockInputStreams & executeWithoutUnion(bool force_no_union);
/** Выполнить запрос, записать результат в нужном формате в buf.
* BlockInputStreamPtr возвращается, чтобы можно было потом получить информацию о плане выполнения запроса.
@ -142,6 +146,9 @@ private:
StoragePtr storage;
IStorage::TableStructureReadLockPtr table_lock;
/// См. описание функции executeWithoutUnion().
bool force_no_union = false;
Logger * log;
};

View File

@ -19,7 +19,7 @@ namespace DB
/** Структура данных для реализации JOIN-а.
* По сути, хэш-таблица: ключи -> строки присоединяемой таблицы.
*
* JOIN-ы бывают четырёх типов: ANY/ALL x LEFT/INNER.
* JOIN-ы бывают восьми типов: ANY/ALL x LEFT/INNER/RIGHT/FULL.
*
* Если указано ANY - выбрать из "правой" таблицы только одну, первую попавшуюся строку, даже если там более одной соответствующей строки.
* Если указано ALL - обычный вариант JOIN-а, при котором строки могут размножаться по числу соответствующих строк "правой" таблицы.
@ -27,6 +27,12 @@ namespace DB
*
* Если указано INNER - оставить только строки, для которых есть хотя бы одна строка "правой" таблицы.
* Если указано LEFT - в случае, если в "правой" таблице нет соответствующей строки, заполнить её значениями "по-умолчанию".
* Если указано RIGHT - выполнить так же, как INNER, запоминая те строки из правой таблицы, которые были присоединены,
* в конце добавить строки из правой таблицы, которые не были присоединены, подставив в качестве значений для левой таблицы, значения "по-умолчанию".
* Если указано FULL - выполнить так же, как LEFT, запоминая те строки из правой таблицы, которые были присоединены,
* в конце добавить строки из правой таблицы, которые не были присоединены, подставив в качестве значений для левой таблицы, значения "по-умолчанию".
*
* То есть, LEFT и RIGHT JOIN-ы не являются симметричными с точки зрения реализации.
*
* Все соединения делаются по равенству кортежа столбцов "ключей" (эквисоединение).
* Неравенства и прочие условия не поддерживаются.
@ -77,11 +83,26 @@ public:
*/
void joinBlock(Block & block) const;
/** Запомнить тотальные значения для последующего использования.
*/
void setTotals(const Block & block) { totals = block; }
bool hasTotals() const { return totals; };
void joinTotals(Block & block) const;
/** Для RIGHT и FULL JOIN-ов.
* Поток, в котором значения по-умолчанию из левой таблицы соединены с неприсоединёнными ранее строками из правой таблицы.
* Использовать только после того, как были сделаны все вызовы joinBlock.
*/
BlockInputStreamPtr createStreamWithNonJoinedRows(Block & left_sample_block, size_t max_block_size) const;
/// Считает суммарное число ключей во всех Join'ах
size_t getTotalRowCount() const;
/// Считает суммарный размер в байтах буфферов всех Join'ов + размер string_pool'а
size_t getTotalByteCount() const;
ASTJoin::Kind getKind() const { return kind; }
/// Ссылка на строку в блоке.
struct RowRef
@ -102,40 +123,65 @@ public:
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {}
};
/** Добавляет или не добавляет флаг - был ли элемент использован.
* Для реализации RIGHT и FULL JOIN-ов.
* NOTE: Можно сохранять флаг в один из бит указателя block или номера row_num.
*/
template <bool enable, typename Base>
struct WithUsedFlag;
template <typename Base>
struct WithUsedFlag<true, Base> : Base
{
mutable bool used = false;
using Base::Base;
using Base_t = Base;
void setUsed() const { used = true; } /// Может выполняться из разных потоков.
bool getUsed() const { return used; }
};
template <typename Base>
struct WithUsedFlag<false, Base> : Base
{
using Base::Base;
using Base_t = Base;
void setUsed() const {}
bool getUsed() const { return true; }
};
/** Разные структуры данных, которые могут использоваться для соединения.
*/
struct MapsAny
template <typename Mapped>
struct MapsTemplate
{
/// Специализация для случая, когда есть один числовой ключ.
typedef HashMap<UInt64, RowRef, HashCRC32<UInt64>> MapUInt64;
typedef HashMap<UInt64, Mapped, HashCRC32<UInt64>> MapUInt64;
/// Специализация для случая, когда есть один строковый ключ.
typedef HashMapWithSavedHash<StringRef, RowRef> MapString;
typedef HashMapWithSavedHash<StringRef, Mapped> MapString;
/** Сравнивает 128 битные хэши.
* Если все ключи фиксированной длины, влезающие целиком в 128 бит, то укладывает их без изменений в 128 бит.
* Иначе - вычисляет SipHash от набора из всех ключей.
* (При этом, строки, содержащие нули посередине, могут склеиться.)
*/
typedef HashMap<UInt128, RowRef, UInt128HashCRC32> MapHashed;
typedef HashMap<UInt128, Mapped, UInt128HashCRC32> MapHashed;
std::unique_ptr<MapUInt64> key64;
std::unique_ptr<MapString> key_string;
std::unique_ptr<MapHashed> hashed;
};
struct MapsAll
{
typedef HashMap<UInt64, RowRefList, HashCRC32<UInt64>> MapUInt64;
typedef HashMapWithSavedHash<StringRef, RowRefList> MapString;
typedef HashMap<UInt128, RowRefList, UInt128HashCRC32> MapHashed;
std::unique_ptr<MapUInt64> key64;
std::unique_ptr<MapString> key_string;
std::unique_ptr<MapHashed> hashed;
};
using MapsAny = MapsTemplate<WithUsedFlag<false, RowRef>>;
using MapsAll = MapsTemplate<WithUsedFlag<false, RowRefList>>;
using MapsAnyFull = MapsTemplate<WithUsedFlag<true, RowRef>>;
using MapsAllFull = MapsTemplate<WithUsedFlag<true, RowRefList>>;
private:
friend class NonJoinedBlockInputStream;
ASTJoin::Kind kind;
ASTJoin::Strictness strictness;
@ -148,8 +194,10 @@ private:
*/
BlocksList blocks;
MapsAny maps_any;
MapsAll maps_all;
MapsAny maps_any; /// Для ANY LEFT|INNER JOIN
MapsAll maps_all; /// Для ALL LEFT|INNER JOIN
MapsAnyFull maps_any_full; /// Для ANY RIGHT|FULL JOIN
MapsAllFull maps_all_full; /// Для ALL RIGHT|FULL JOIN
/// Дополнительные данные - строки, а также продолжения односвязных списков строк.
Arena pool;
@ -176,6 +224,8 @@ private:
size_t max_bytes;
OverflowMode overflow_mode;
Block totals;
/** Защищает работу с состоянием в функциях insertFromBlock и joinBlock.
* Эти функции могут вызываться одновременно из разных потоков только при использовании StorageJoin,
* и StorageJoin вызывает только эти две функции.

View File

@ -22,46 +22,49 @@ namespace DB
* Также реализует ограничение на их количество.
*/
/// Запрос и данные о его выполнении.
struct ProcessListElement
{
String query;
String user;
String query_id;
Poco::Net::IPAddress ip_address;
Stopwatch watch;
Progress progress;
MemoryTracker memory_tracker;
bool is_cancelled = false;
ProcessListElement(const String & query_, const String & user_,
const String & query_id_, const Poco::Net::IPAddress & ip_address_,
size_t max_memory_usage)
: query(query_), user(user_), query_id(query_id_), ip_address(ip_address_), memory_tracker(max_memory_usage)
{
current_memory_tracker = &memory_tracker;
}
~ProcessListElement()
{
current_memory_tracker = nullptr;
}
bool update(const Progress & value)
{
progress.incrementPiecewiseAtomically(value);
return !is_cancelled;
}
};
class ProcessList
{
friend class Entry;
public:
/// Запрос и данные о его выполнении.
struct Element
{
String query;
String user;
String query_id;
Poco::Net::IPAddress ip_address;
Stopwatch watch;
Progress progress;
MemoryTracker memory_tracker;
bool is_cancelled = false;
Element(const String & query_, const String & user_,
const String & query_id_, const Poco::Net::IPAddress & ip_address_,
size_t max_memory_usage)
: query(query_), user(user_), query_id(query_id_), ip_address(ip_address_), memory_tracker(max_memory_usage)
{
current_memory_tracker = &memory_tracker;
}
~Element()
{
current_memory_tracker = nullptr;
}
bool update(const Progress & value)
{
progress.incrementPiecewiseAtomically(value);
return !is_cancelled;
}
};
using Element = ProcessListElement;
/// list, чтобы итераторы не инвалидировались. NOTE: можно заменить на cyclic buffer, но почти незачем.
typedef std::list<Element> Containter;

View File

@ -2,6 +2,7 @@
#include <string.h>
#include <unordered_map>
#include <memory>
#include <Poco/Timespan.h>
@ -129,7 +130,7 @@ public:
{
return cont.empty();
}
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
/// Обновляет максимальные значения значениями из quota.
@ -154,7 +155,7 @@ public:
}
};
typedef Poco::SharedPtr<QuotaForIntervals> QuotaForIntervalsPtr;
typedef std::shared_ptr<QuotaForIntervals> QuotaForIntervalsPtr;
/// Ключ квоты -> квоты за интервалы. Если квота не допускает ключей, то накопленные значения хранятся по ключу 0.

View File

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

View File

@ -1,7 +1,6 @@
#pragma once
#include <DB/Core/QueryProcessingStage.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Interpreters/InterpreterQuery.h>

View File

@ -51,7 +51,7 @@ public:
bool detach = false; /// true для DETACH PARTITION.
bool part = false; /// true для ATTACH [UNREPLICATED] PART
bool unreplicated = false; /// true для ATTACH UNREPLICATED ...
bool unreplicated = false; /// true для ATTACH UNREPLICATED, DROP UNREPLICATED ...
/** Для FETCH PARTITION - путь в ZK к шарду, с которого скачивать партицию.
*/

View File

@ -30,7 +30,9 @@ public:
enum Kind
{
Inner, /// Оставить только записи, для которых в "правой" таблице есть соответствующая.
Left /// Если в "правой" таблице нет соответствующих записей, заполнить столбцы значениями "по-умолчанию".
Left, /// Если в "правой" таблице нет соответствующих записей, заполнить столбцы значениями "по-умолчанию".
Right,
Full
};
Locality locality = Local;
@ -54,7 +56,13 @@ public:
writeString("Global", wb);
writeString(strictness == Any ? "Any" : "All", wb);
writeString(kind == Inner ? "Inner" : "Left", wb);
writeString(
kind == Inner ? "Inner"
: (kind == Left ? "Left"
: (kind == Right ? "Right"
: "Full")), wb);
writeString("Join", wb);
}

View File

@ -1,12 +1,7 @@
#pragma once
#include <list>
#include <Poco/SharedPtr.h>
#include <DB/Core/Types.h>
#include <DB/Parsers/IParser.h>
#include <iostream>
namespace DB
{
@ -16,27 +11,7 @@ namespace DB
class IParserBase : public IParser
{
public:
bool parse(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos new_max_parsed_pos = pos;
Expected new_expected = getName();
bool res = parseImpl(pos, end, node, new_max_parsed_pos, new_expected);
if (new_max_parsed_pos > max_parsed_pos)
max_parsed_pos = new_max_parsed_pos;
if (new_max_parsed_pos >= max_parsed_pos)
expected = new_expected;
if (!res)
node = nullptr;
if (pos > end)
throw Exception("Logical error: pos > end.", ErrorCodes::LOGICAL_ERROR);
return res;
}
bool parse(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
protected:
virtual bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) = 0;

View File

@ -83,7 +83,6 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & nod
return true;
}
pos = begin;
return false;
}
@ -119,10 +118,6 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
ParserTernaryOperatorExpression expr_parser;
const auto begin = pos;
const auto reset_pos_and_return = [&pos, begin] {
pos = begin;
return false;
};
/// mandatory column name
ASTPtr name;
@ -160,10 +155,10 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
ws.ignore(pos, end, max_parsed_pos, expected);
if (!expr_parser.parse(pos, end, default_expression, max_parsed_pos, expected))
return reset_pos_and_return();
return false;
}
else if (!type)
return reset_pos_and_return(); /// reject sole column name without type
return false; /// reject sole column name without type
const auto column_declaration = new ASTColumnDeclaration{StringRange{begin, pos}};
node = column_declaration;

View File

@ -9,7 +9,7 @@ namespace DB
/// Распарсить запрос или записать сообщение об ошибке в out_error_message.
ASTPtr tryParseQuery(
IParser & parser,
IParser::Pos begin,
IParser::Pos & pos, /// Сдвигается до конца распарсенного фрагмента.
IParser::Pos end,
std::string & out_error_message,
bool hilite,
@ -17,6 +17,13 @@ ASTPtr tryParseQuery(
/// Распарсить запрос или кинуть исключение с сообщением об ошибке.
ASTPtr parseQueryAndMovePosition(
IParser & parser,
IParser::Pos & pos, /// Сдвигается до конца распарсенного фрагмента.
IParser::Pos end,
const std::string & description);
ASTPtr parseQuery(
IParser & parser,
IParser::Pos begin,

View File

@ -0,0 +1,97 @@
#include <DB/IO/CompressedStream.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/Core/ErrorCodes.h>
#include <statdaemons/Exception.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
/** Позволяет выбрать метод сжатия по указанным в конфигурационном файле условиям.
* Конфиг выглядит примерно так:
<compression>
<!-- Набор вариантов. Варианты проверяются подряд. Побеждает последний сработавший вариант. Если ни один не сработал, то используется lz4. -->
<case>
<!-- Условия. Должны сработать одновременно все. Часть условий могут быть не указаны. -->
<min_part_size>10000000000</min_part_size> <!-- Минимальный размер куска в байтах. -->
<min_part_size_ratio>0.01</min_part_size_ratio> <!-- Минимальный размер куска относительно всех данных таблицы. -->
<!-- Какой метод сжатия выбрать. -->
<method>zstd</method>
</case>
<case>
...
</case>
</compression>
*/
class CompressionMethodSelector
{
private:
struct Element
{
size_t min_part_size = 0;
double min_part_size_ratio = 0;
CompressionMethod method = CompressionMethod::LZ4;
void setMethod(const std::string & name)
{
if (name == "lz4")
method = CompressionMethod::LZ4;
else if (name == "zstd")
method = CompressionMethod::ZSTD;
else
throw Exception("Unknown compression method " + name, ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
}
Element(Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
min_part_size = parse<size_t>(config.getString(config_prefix + ".min_part_size", "0"));
min_part_size_ratio = config.getDouble(config_prefix + ".min_part_size_ratio", 0);
setMethod(config.getString(config_prefix + ".method"));
}
bool check(size_t part_size, double part_size_ratio) const
{
return part_size >= min_part_size
&& part_size_ratio >= min_part_size_ratio;
}
};
std::vector<Element> elements;
public:
CompressionMethodSelector() {} /// Всегда возвращает метод по-умолчанию.
CompressionMethodSelector(Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
for (const auto & name : keys)
{
if (0 != strncmp(name.data(), "case", strlen("case")))
throw Exception("Unknown element in config: " + config_prefix + "." + name + ", must be 'case'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
elements.emplace_back(config, config_prefix + "." + name);
}
}
CompressionMethod choose(size_t part_size, double part_size_ratio) const
{
CompressionMethod res = CompressionMethod::LZ4;
for (const auto & element : elements)
if (element.check(part_size, part_size_ratio))
res = element.method;
return res;
}
};
}

View File

@ -3,10 +3,17 @@
#include <DB/DataStreams/RemoteBlockOutputStream.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/Storages/StorageDistributed.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <boost/algorithm/string/find_iterator.hpp>
#include <boost/algorithm/string/finder.hpp>
#include <Poco/DirectoryIterator.h>
#include <thread>
#include <mutex>
#include <condition_variable>
namespace DB
{

View File

@ -8,6 +8,7 @@
#include <DB/IO/CompressedWriteBuffer.h>
#include <DB/DataStreams/NativeBlockOutputStream.h>
#include <DB/Interpreters/InterpreterInsertQuery.h>
#include <DB/Interpreters/Cluster.h>
#include <statdaemons/Increment.h>
#include <memory>
@ -16,6 +17,7 @@
#include <iostream>
#include <type_traits>
namespace DB
{

View File

@ -207,7 +207,7 @@ public:
/** Выполнить запрос (DROP|DETACH) PARTITION.
*/
virtual void dropPartition(const Field & partition, bool detach, const Settings & settings)
virtual void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings)
{
throw Exception("Method dropPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -236,9 +236,9 @@ public:
/** Выполнить какую-либо фоновую работу. Например, объединение кусков в таблице типа MergeTree.
* Возвращает - была ли выполнена какая-либо работа.
*/
bool optimize(size_t aio_threshold = 0)
virtual bool optimize(const Settings & settings)
{
return performOptimize(aio_threshold);
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Получить запрос CREATE TABLE, который описывает данную таблицу.
@ -279,12 +279,6 @@ public:
/// проверяет валидность данных
virtual bool checkData() const { throw DB::Exception("Check query is not supported for " + getName() + " storage"); }
protected:
virtual bool performOptimize(size_t aio_threshold)
{
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
protected:
using ITableDeclaration::ITableDeclaration;

View File

@ -1,5 +1,7 @@
#pragma once
#include <memory>
#include <DB/Common/LRUCache.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/SipHash.h>
@ -32,7 +34,7 @@ struct MarkInCompressedFile
typedef std::vector<MarkInCompressedFile> MarksInCompressedFile;
/// Оценка количества байт, занимаемых засечками в кеше.
/// Оценка количества байтов, занимаемых засечками в кеше.
struct MarksWeightFunction
{
size_t operator()(const MarksInCompressedFile & marks) const
@ -79,6 +81,6 @@ public:
}
};
typedef Poco::SharedPtr<MarkCache> MarkCachePtr;
typedef std::shared_ptr<MarkCache> MarkCachePtr;
}

View File

@ -10,31 +10,52 @@ namespace DB
{
struct MergeInfo
{
const std::string database;
const std::string table;
const std::string result_part_name;
Stopwatch watch;
Float64 progress{};
std::uint64_t num_parts{};
std::uint64_t total_size_bytes_compressed{};
std::uint64_t total_size_marks{};
std::uint64_t bytes_read_uncompressed{};
std::uint64_t rows_read{};
std::uint64_t bytes_written_uncompressed{};
std::uint64_t rows_written{};
MergeInfo(const std::string & database, const std::string & table, const std::string & result_part_name)
: database{database}, table{table}, result_part_name{result_part_name}
{
}
};
class MergeList;
class MergeListEntry
{
MergeList & list;
using container_t = std::list<MergeInfo>;
container_t::iterator it;
public:
MergeListEntry(const MergeListEntry &) = delete;
MergeListEntry & operator=(const MergeListEntry &) = delete;
MergeListEntry(MergeList & list, const container_t::iterator it) : list(list), it{it} {}
~MergeListEntry();
MergeInfo * operator->() { return &*it; }
};
class MergeList
{
friend class Entry;
struct MergeInfo
{
const std::string database;
const std::string table;
const std::string result_part_name;
Stopwatch watch;
Float64 progress{};
std::uint64_t num_parts{};
std::uint64_t total_size_bytes_compressed{};
std::uint64_t total_size_marks{};
std::uint64_t bytes_read_uncompressed{};
std::uint64_t rows_read{};
std::uint64_t bytes_written_uncompressed{};
std::uint64_t rows_written{};
MergeInfo(const std::string & database, const std::string & table, const std::string & result_part_name)
: database{database}, table{table}, result_part_name{result_part_name}
{
}
};
friend class MergeListEntry;
using container_t = std::list<MergeInfo>;
@ -42,25 +63,7 @@ class MergeList
container_t merges;
public:
class Entry
{
MergeList & list;
container_t::iterator it;
public:
Entry(const Entry &) = delete;
Entry & operator=(const Entry &) = delete;
Entry(MergeList & list, const container_t::iterator it) : list(list), it{it} {}
~Entry()
{
std::lock_guard<std::mutex> lock{list.mutex};
list.merges.erase(it);
}
MergeInfo * operator->() { return &*it; }
};
using Entry = MergeListEntry;
using EntryPtr = std::unique_ptr<Entry>;
template <typename... Args>
@ -78,4 +81,11 @@ public:
};
inline MergeListEntry::~MergeListEntry()
{
std::lock_guard<std::mutex> lock{list.mutex};
list.merges.erase(it);
}
}

View File

@ -182,15 +182,18 @@ protected:
if (!reader)
{
UncompressedCache * uncompressed_cache = use_uncompressed_cache ? storage.context.getUncompressedCache() : nullptr;
if (use_uncompressed_cache)
owned_uncompressed_cache = storage.context.getUncompressedCache();
owned_mark_cache = storage.context.getMarkCache();
reader.reset(new MergeTreeReader(
path, owned_data_part, columns, uncompressed_cache, storage,
path, owned_data_part, columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), storage,
all_mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size));
if (prewhere_actions)
pre_reader.reset(new MergeTreeReader(
path, owned_data_part, pre_columns, uncompressed_cache, storage,
path, owned_data_part, pre_columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), storage,
all_mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size));
}
@ -384,6 +387,9 @@ private:
size_t min_bytes_to_use_direct_io;
size_t max_read_buffer_size;
UncompressedCachePtr owned_uncompressed_cache;
MarkCachePtr owned_mark_cache;
};
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/Storages/StorageMergeTree.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <iomanip>
namespace DB

View File

@ -9,7 +9,9 @@
#include <DB/Storages/MergeTree/ActiveDataPartSet.h>
#include <DB/IO/ReadBufferFromString.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/Common/SipHash.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <Poco/RWLock.h>
@ -301,7 +303,7 @@ public:
}
}
/// Вычисляем сумарный размер всей директории со всеми файлами
/// Вычисляем суммарный размер всей директории со всеми файлами
static size_t calcTotalSize(const String & from)
{
Poco::File cur(from);
@ -715,6 +717,10 @@ public:
DataPartsVector getDataPartsVector();
DataParts getAllDataParts();
/** Размер активной части в количестве байт.
*/
size_t getTotalActiveSizeInBytes();
/** Максимальное количество кусков в одном месяце.
*/
size_t getMaxPartsCountForMonth();

View File

@ -7,6 +7,9 @@
namespace DB
{
class MergeListEntry;
/** Умеет выбирать куски для слияния и сливать их.
*/
class MergeTreeDataMerger
@ -41,7 +44,7 @@ public:
* приблизительно пропорционально количеству уже выписанных данных.
*/
MergeTreeData::DataPartPtr mergeParts(
const MergeTreeData::DataPartsVector & parts, const String & merged_name, MergeList::Entry & merge_entry,
const MergeTreeData::DataPartsVector & parts, const String & merged_name, MergeListEntry & merge_entry,
size_t aio_threshold, MergeTreeData::Transaction * out_transaction = nullptr,
DiskSpaceMonitor::Reservation * disk_reservation = nullptr);

View File

@ -1,5 +1,6 @@
#pragma once
#include <DB/Storages/MarkCache.h>
#include <DB/Storages/MergeTree/MergeTreeData.h>
#include <DB/DataTypes/IDataType.h>
#include <DB/DataTypes/DataTypeNested.h>
@ -40,10 +41,12 @@ class MergeTreeReader
public:
MergeTreeReader(const String & path_, /// Путь к куску
const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_,
bool use_uncompressed_cache_, MergeTreeData & storage_, const MarkRanges & all_mark_ranges,
UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_,
MergeTreeData & storage_, const MarkRanges & all_mark_ranges,
size_t aio_threshold_, size_t max_read_buffer_size_)
: path(path_), data_part(data_part), part_name(data_part->name), columns(columns_),
use_uncompressed_cache(use_uncompressed_cache_), storage(storage_), all_mark_ranges(all_mark_ranges),
uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_),
storage(storage_), all_mark_ranges(all_mark_ranges),
aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_)
{
try
@ -256,16 +259,36 @@ private:
(*marks)[right].offset_in_compressed_file - (*marks)[all_mark_ranges[i].begin].offset_in_compressed_file);
}
size_t buffer_size = max_read_buffer_size < max_mark_range ? max_read_buffer_size : max_mark_range;
size_t buffer_size = std::min(max_read_buffer_size, max_mark_range);
size_t estimated_size = 0;
if (aio_threshold > 0)
{
for (const auto & mark_range : all_mark_ranges)
{
size_t offset_begin = (*marks)[mark_range.begin].offset_in_compressed_file;
size_t offset_end;
if (mark_range.end < (*marks).size())
offset_end = (*marks)[mark_range.end].offset_in_compressed_file;
else
offset_end = Poco::File(path_prefix + ".bin").getSize();
if (offset_end > 0)
estimated_size += offset_end - offset_begin;
}
}
if (uncompressed_cache)
{
cached_buffer = new CachedCompressedReadBuffer(path_prefix + ".bin", uncompressed_cache, aio_threshold, buffer_size);
cached_buffer = new CachedCompressedReadBuffer(path_prefix + ".bin", uncompressed_cache,
estimated_size, aio_threshold, buffer_size);
data_buffer = &*cached_buffer;
}
else
{
non_cached_buffer = new CompressedReadBufferFromFile(path_prefix + ".bin", aio_threshold, buffer_size);
non_cached_buffer = new CompressedReadBufferFromFile(path_prefix + ".bin", estimated_size,
aio_threshold, buffer_size);
data_buffer = &*non_cached_buffer;
}
}
@ -315,7 +338,7 @@ private:
{
/// Более хорошая диагностика.
if (e.code() == ErrorCodes::ARGUMENT_OUT_OF_BOUND)
throw Exception(e.message() + " (while seeking to mark " + Poco::NumberFormatter::format(index)
throw Exception(e.message() + " (while seeking to mark " + toString(index)
+ " of column " + path_prefix + "; offsets are: "
+ toString(mark.offset_in_compressed_file) + " "
+ toString(mark.offset_in_decompressed_block) + ")", e.code());
@ -336,7 +359,9 @@ private:
NamesAndTypesList columns;
const NameAndTypePair * added_minimum_size_column = nullptr;
bool use_uncompressed_cache;
UncompressedCache * uncompressed_cache;
MarkCache * mark_cache;
MergeTreeData & storage;
const MarkRanges & all_mark_ranges;
size_t aio_threshold;
@ -352,9 +377,6 @@ private:
if (!Poco::File(path + escaped_column_name + ".bin").exists())
return;
UncompressedCache * uncompressed_cache = use_uncompressed_cache ? storage.context.getUncompressedCache() : NULL;
MarkCache * mark_cache = storage.context.getMarkCache();
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{

View File

@ -9,6 +9,7 @@
#include <DB/Common/escapeForFileName.h>
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataStreams/IBlockOutputStream.h>
namespace DB

View File

@ -2,6 +2,7 @@
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <DB/DataStreams/IBlockOutputStream.h>
namespace DB
@ -11,7 +12,7 @@ class ReplicatedMergeTreeBlockOutputStream : public IBlockOutputStream
{
public:
ReplicatedMergeTreeBlockOutputStream(StorageReplicatedMergeTree & storage_, const String & insert_id_)
: storage(storage_), insert_id(insert_id_), block_index(0),
: storage(storage_), insert_id(insert_id_),
log(&Logger::get(storage.data.getLogName() + " (Replicated OutputStream)")) {}
void writePrefix() override
@ -145,7 +146,7 @@ public:
private:
StorageReplicatedMergeTree & storage;
String insert_id;
size_t block_index;
size_t block_index = 0;
Logger * log;

View File

@ -70,6 +70,7 @@ public:
/// Сбрасывает все буферы в подчинённую таблицу.
void shutdown() override;
bool optimize(const Settings & settings) override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
@ -123,8 +124,6 @@ private:
void writeBlockToDestination(const Block & block, StoragePtr table);
void flushThread();
bool performOptimize(size_t aio_threshold) override;
};
}

View File

@ -4,6 +4,7 @@
#include <DB/Interpreters/Context.h>
#include <DB/Storages/IStorage.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
namespace DB
@ -71,7 +72,7 @@ private:
class MergeTask;
using MergeTaskPtr = std::shared_ptr<MergeTask>;
MergeTaskPtr merge_task;
DB::BackgroundProcessingPool::TaskHandle merge_task_handle;
BackgroundProcessingPool::TaskHandle merge_task_handle;
StorageChunkMerger(
const std::string & this_database_,

View File

@ -5,6 +5,7 @@
#include <DB/Client/ConnectionPoolWithFailover.h>
#include <DB/Interpreters/Settings.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/ExpressionActions.h>
namespace DB

View File

@ -12,7 +12,7 @@ namespace DB
* а также записаны в файл-бэкап, для восстановления после перезапуска.
* Чтение из таблицы напрямую невозможно - возможно лишь указание в правой части JOIN.
*
* При использовании, JOIN должен быть соответствующего типа (ANY|ALL LEFT|INNER).
* При использовании, JOIN должен быть соответствующего типа (ANY|ALL LEFT|INNER ...).
*/
class StorageJoin : public StorageSetOrJoinBase
{
@ -44,7 +44,7 @@ public:
private:
const Names & key_names;
ASTJoin::Kind kind; /// LEFT | INNER
ASTJoin::Kind kind; /// LEFT | INNER ...
ASTJoin::Strictness strictness; /// ANY | ALL
JoinPtr join;

View File

@ -32,6 +32,7 @@ public:
BlockOutputStreamPtr write(ASTPtr query) override;
void drop() override;
bool optimize(const Settings & settings) override;
BlockInputStreams read(
const Names & column_names,
@ -55,8 +56,6 @@ private:
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach_);
bool performOptimize(size_t aio_threshold) override;
};
}

View File

@ -5,6 +5,8 @@
#include <DB/Storages/MergeTree/MergeTreeDataWriter.h>
#include <DB/Storages/MergeTree/MergeTreeDataMerger.h>
#include <DB/Storages/MergeTree/DiskSpaceMonitor.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
namespace DB
{
@ -79,7 +81,14 @@ public:
BlockOutputStreamPtr write(ASTPtr query) override;
void dropPartition(const Field & partition, bool detach, const Settings & settings) override;
/** Выполнить очередной шаг объединения кусков.
*/
bool optimize(const Settings & settings) override
{
return merge(settings.min_bytes_to_use_direct_io, true);
}
void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings) override;
void attachPartition(const Field & partition, bool unreplicated, bool part, const Settings & settings) override;
void freezePartition(const Field & partition, const Settings & settings) override;
@ -93,14 +102,6 @@ public:
MergeTreeData & getData() { return data; }
private:
/** Выполнить очередной шаг объединения кусков.
*/
bool performOptimize(size_t aio_threshold) override
{
return merge(aio_threshold, true);
}
private:
String path;
String database_name;

View File

@ -10,6 +10,7 @@
#include <DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
#include <DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
#include <DB/Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <zkutil/ZooKeeper.h>
#include <zkutil/LeaderElection.h>
@ -83,9 +84,11 @@ public:
BlockOutputStreamPtr write(ASTPtr query) override;
bool optimize(const Settings & settings) override;
void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context) override;
void dropPartition(const Field & partition, bool detach, const Settings & settings) override;
void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings) override;
void attachPartition(const Field & partition, bool unreplicated, bool part, const Settings & settings) override;
void fetchPartition(const Field & partition, const String & from, const Settings & settings) override;
void freezePartition(const Field & partition, const Settings & settings) override;
@ -131,6 +134,8 @@ public:
void getStatus(Status & res, bool with_zk_fields = true);
private:
void dropUnreplicatedPartition(const Field & partition, const Settings & settings);
friend class ReplicatedMergeTreeBlockOutputStream;
friend class ReplicatedMergeTreeRestartingThread;
friend class ReplicatedMergeTreeCleanupThread;
@ -386,8 +391,6 @@ private:
*/
void waitForReplicaToProcessLogEntry(const String & replica_name, const LogEntry & entry);
bool performOptimize(size_t aio_threshold) override;
/// Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper.
static String padIndex(UInt64 index)
{

View File

@ -5,6 +5,7 @@
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/DataStreams/RemoteBlockInputStream.h>
#include <DB/Interpreters/reinterpretAsIdentifier.h>
#include <DB/Interpreters/Cluster.h>
struct data;

View File

@ -1,5 +1,3 @@
#define DBMS_CLIENT 1 /// Используется в Context.h
#include <unistd.h>
#include <stdlib.h>
#include <fcntl.h>
@ -33,9 +31,6 @@
#include <DB/DataStreams/RemoteBlockInputStream.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Interpreters/Context.h>
#include <DB/Client/Connection.h>

View File

@ -1,5 +1,3 @@
#define DBMS_CLIENT 1 /// Используется в Context.h
#include <unistd.h>
#include <stdlib.h>
#include <fcntl.h>
@ -653,7 +651,7 @@ private:
}
}
else
res = DB::parseQuery(parser, pos, end, "");
res = DB::parseQueryAndMovePosition(parser, pos, end, "");
if (is_interactive)
{

View File

@ -13,6 +13,10 @@ Block AggregatingBlockInputStream::readImpl()
{
executed = true;
AggregatedDataVariants data_variants;
Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); };
aggregator.setCancellationHook(hook);
aggregator.execute(children.back(), data_variants);
blocks = aggregator.convertToBlocks(data_variants, final, 1);
it = blocks.begin();

View File

@ -1,5 +1,6 @@
#include <DB/Interpreters/Set.h>
#include <DB/Interpreters/Join.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <DB/DataStreams/CreatingSetsBlockInputStream.h>
#include <iomanip>
@ -31,6 +32,18 @@ Block CreatingSetsBlockInputStream::readImpl()
return children.back()->read();
}
const Block & CreatingSetsBlockInputStream::getTotals()
{
auto input = dynamic_cast<IProfilingBlockInputStream *>(children.back().get());
if (input)
return input->getTotals();
else
return totals;
}
void CreatingSetsBlockInputStream::create(SubqueryForSet & subquery)
{
LOG_TRACE(log, (subquery.set ? "Creating set. " : "")
@ -121,8 +134,13 @@ void CreatingSetsBlockInputStream::create(SubqueryForSet & subquery)
size_t head_rows = 0;
if (IProfilingBlockInputStream * profiling_in = dynamic_cast<IProfilingBlockInputStream *>(&*subquery.source))
{
head_rows = profiling_in->getInfo().rows;
if (subquery.join)
subquery.join->setTotals(profiling_in->getTotals());
}
if (rows != 0)
{
std::stringstream msg;

View File

@ -13,6 +13,10 @@ Block MergingAggregatedBlockInputStream::readImpl()
{
executed = true;
AggregatedDataVariants data_variants;
Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); };
aggregator.setCancellationHook(hook);
aggregator.mergeStream(children.back(), data_variants, max_threads);
blocks = aggregator.convertToBlocks(data_variants, final, max_threads);
it = blocks.begin();

View File

@ -13,7 +13,7 @@ namespace DB
{
static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows)
void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows)
{
/** Для массивов требуется сначала десериализовать смещения, а потом значения.
*/

View File

@ -15,19 +15,50 @@ namespace DB
{
static void writeData(const IDataType & type, const IColumn & column, WriteBuffer & ostr)
void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit)
{
/** Если есть столбцы-константы - то материализуем их.
* (Так как тип данных не умеет сериализовывать/десериализовывать константы.)
*/
ColumnPtr full_column = column->isConst()
? static_cast<const IColumnConst &>(*column).convertToFullColumn()
: column;
/** Для массивов требуется сначала сериализовать смещения, а потом значения.
*/
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
type_arr->getOffsetsType()->serializeBinary(*typeid_cast<const ColumnArray &>(column).getOffsetsColumn(), ostr);
const ColumnArray & column_array = typeid_cast<const ColumnArray &>(*full_column);
type_arr->getOffsetsType()->serializeBinary(*column_array.getOffsetsColumn(), ostr, offset, limit);
if (!typeid_cast<const ColumnArray &>(column).getData().empty())
writeData(*type_arr->getNestedType(), typeid_cast<const ColumnArray &>(column).getData(), ostr);
if (!typeid_cast<const ColumnArray &>(*full_column).getData().empty())
{
const ColumnArray::Offsets_t & offsets = column_array.getOffsets();
if (offset > offsets.size())
return;
/** offset - с какого массива писать.
* limit - сколько массивов максимум записать, или 0, если писать всё, что есть.
* end - до какого массива заканчивается записываемый кусок.
*
* nested_offset - с какого элемента внутренностей писать.
* nested_limit - сколько элементов внутренностей писать, или 0, если писать всё, что есть.
*/
size_t end = std::min(offset + limit, offsets.size());
size_t nested_offset = offset ? offsets[offset - 1] : 0;
size_t nested_limit = limit
? offsets[end - 1] - nested_offset
: 0;
if (limit == 0 || nested_limit)
writeData(*type_arr->getNestedType(), typeid_cast<const ColumnArray &>(*full_column).getDataPtr(), ostr, nested_offset, nested_limit);
}
}
else
type.serializeBinary(column, ostr);
type.serializeBinary(*full_column, ostr, offset, limit);
}
@ -54,15 +85,7 @@ void NativeBlockOutputStream::write(const Block & block)
writeStringBinary(column.type->getName(), ostr);
/// Данные
/** Если есть столбцы-константы - то материализуем их.
* (Так как тип данных не умеет сериализовывать/десериализовывать константы.)
*/
ColumnPtr col = column.column->isConst()
? static_cast<const IColumnConst &>(*column.column).convertToFullColumn()
: column.column;
writeData(*column.type, *col, ostr);
writeData(*column.type, column.column, ostr, 0, 0);
}
}

View File

@ -1,5 +1,6 @@
#include <iostream>
#include <iomanip>
#include <thread>
#include <Poco/SharedPtr.h>
#include <Poco/Stopwatch.h>
@ -13,6 +14,7 @@
#include <DB/DataStreams/FilterBlockInputStream.h>
#include <DB/DataStreams/TabSeparatedRowOutputStream.h>
#include <DB/DataStreams/ForkBlockInputStreams.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/DataStreams/copyData.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>

View File

@ -8,6 +8,7 @@
#include <DB/Interpreters/loadMetadata.h>
#include <DB/Interpreters/executeQuery.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/DataStreams/glueBlockInputStreams.h>

View File

@ -18,6 +18,7 @@
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/parseQuery.h>
namespace DB
{
@ -69,15 +70,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const
Array params_row;
ParserExpressionList args_parser;
ASTPtr args_ast;
Expected expected = "";
IParser::Pos pos = parameters.data();
IParser::Pos end = pos + parameters.size();
IParser::Pos max_parsed_pos = pos;
if (!(args_parser.parse(pos, end, args_ast, max_parsed_pos, expected) && pos == end))
throw Exception("Cannot parse parameters for data type " + name, ErrorCodes::SYNTAX_ERROR);
ASTPtr args_ast = parseQuery(args_parser, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
ASTExpressionList & args_list = typeid_cast<ASTExpressionList &>(*args_ast);
if (args_list.children.empty())
@ -121,14 +114,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const
if (base_name == "Nested")
{
ParserNameTypePairList columns_p;
ASTPtr columns_ast;
Expected expected = "";
IParser::Pos pos = parameters.data();
IParser::Pos end = pos + parameters.size();
IParser::Pos max_parsed_pos = pos;
if (!(columns_p.parse(pos, end, columns_ast, max_parsed_pos, expected) && pos == end))
throw Exception("Cannot parse parameters for data type " + name, ErrorCodes::SYNTAX_ERROR);
ASTPtr columns_ast = parseQuery(columns_p, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
NamesAndTypesListPtr columns = new NamesAndTypesList;
@ -151,14 +137,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const
if (base_name == "Tuple")
{
ParserExpressionList columns_p;
ASTPtr columns_ast;
Expected expected = "";
IParser::Pos pos = parameters.data();
IParser::Pos end = pos + parameters.size();
IParser::Pos max_parsed_pos = pos;
if (!(columns_p.parse(pos, end, columns_ast, max_parsed_pos, expected) && pos == end))
throw Exception("Cannot parse parameters for data type " + name, ErrorCodes::SYNTAX_ERROR);
ASTPtr columns_ast = parseQuery(columns_p, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
DataTypes elems;

View File

@ -58,7 +58,6 @@ FunctionFactory::FunctionFactory()
}
FunctionPtr FunctionFactory::get(
const String & name,
const Context & context) const

View File

@ -17,6 +17,7 @@ void registerFunctionsHashing(FunctionFactory & factory)
factory.registerFunction<FunctionCityHash64>();
factory.registerFunction<FunctionIntHash32>();
factory.registerFunction<FunctionIntHash64>();
factory.registerFunction<FunctionURLHash>();
}
}

View File

@ -151,6 +151,10 @@ void ReadBufferAIO::synchronousRead()
{
prepare();
bytes_read = ::pread(fd, buffer_begin, region_aligned_size, region_aligned_begin);
ProfileEvents::increment(ProfileEvents::ReadBufferAIORead);
ProfileEvents::increment(ProfileEvents::ReadBufferAIOReadBytes, bytes_read);
finalize();
}
@ -190,6 +194,9 @@ bool ReadBufferAIO::waitForAIOCompletion()
is_pending_read = false;
bytes_read = events[0].res;
ProfileEvents::increment(ProfileEvents::ReadBufferAIORead);
ProfileEvents::increment(ProfileEvents::ReadBufferAIOReadBytes, bytes_read);
return true;
}

View File

@ -175,6 +175,9 @@ bool WriteBufferAIO::waitForAIOCompletion()
is_pending_write = false;
bytes_written = events[0].res;
ProfileEvents::increment(ProfileEvents::WriteBufferAIOWrite);
ProfileEvents::increment(ProfileEvents::WriteBufferAIOWriteBytes, bytes_written);
return true;
}

View File

@ -17,4 +17,4 @@ off_t WriteBufferFromFileBase::seek(off_t off, int whence)
return doSeek(off, whence);
}
}
}

View File

@ -1,20 +1,24 @@
#include <DB/IO/createReadBufferFromFileBase.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/ReadBufferAIO.h>
#include <Poco/File.h>
#include <DB/Common/ProfileEvents.h>
namespace DB
{
ReadBufferFromFileBase * createReadBufferFromFileBase(const std::string & filename_, size_t aio_threshold,
size_t buffer_size_, int flags_, char * existing_memory_, size_t alignment)
ReadBufferFromFileBase * createReadBufferFromFileBase(const std::string & filename_, size_t estimated_size,
size_t aio_threshold, size_t buffer_size_, int flags_, char * existing_memory_, size_t alignment)
{
size_t file_size = (aio_threshold > 0) ? Poco::File(filename_).getSize() : 0;
if ((aio_threshold == 0) || (file_size < aio_threshold))
if ((aio_threshold == 0) || (estimated_size < aio_threshold))
{
ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary);
return new ReadBufferFromFile(filename_, buffer_size_, flags_, existing_memory_, alignment);
}
else
{
ProfileEvents::increment(ProfileEvents::CreatedReadBufferAIO);
return new ReadBufferAIO(filename_, buffer_size_, flags_, existing_memory_);
}
}
}

View File

@ -1,18 +1,25 @@
#include <DB/IO/createWriteBufferFromFileBase.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/WriteBufferAIO.h>
#include <DB/Common/ProfileEvents.h>
namespace DB
{
WriteBufferFromFileBase * createWriteBufferFromFileBase(const std::string & filename_, size_t estimated_size,
size_t aio_threshold, size_t buffer_size_, int flags_, mode_t mode, char * existing_memory_,
size_t aio_threshold, size_t buffer_size_, int flags_, mode_t mode, char * existing_memory_,
size_t alignment)
{
if ((aio_threshold == 0) || (estimated_size < aio_threshold))
{
ProfileEvents::increment(ProfileEvents::CreatedWriteBufferOrdinary);
return new WriteBufferFromFile(filename_, buffer_size_, flags_, mode, existing_memory_, alignment);
}
else
{
ProfileEvents::increment(ProfileEvents::CreatedWriteBufferAIO);
return new WriteBufferAIO(filename_, buffer_size_, flags_, mode, existing_memory_);
}
}
}

View File

@ -25,7 +25,7 @@ int main(int argc, char ** argv)
{
Stopwatch watch;
CachedCompressedReadBuffer in(path, &cache, std::numeric_limits<size_t>::max());
CachedCompressedReadBuffer in(path, &cache, 0, 0);
WriteBufferFromFile out("/dev/null");
copyData(in, out);
@ -37,7 +37,7 @@ int main(int argc, char ** argv)
{
Stopwatch watch;
CachedCompressedReadBuffer in(path, &cache, std::numeric_limits<size_t>::max());
CachedCompressedReadBuffer in(path, &cache, 0, 0);
WriteBufferFromFile out("/dev/null");
copyData(in, out);

View File

@ -14,12 +14,13 @@ namespace
{
void run();
void prepare(size_t s, std::string & directory, std::string & filename, std::string & buf);
void prepare2(std::string & directory, std::string & filename, std::string & buf);
void prepare3(std::string & directory, std::string & filename, std::string & buf);
void prepare4(std::string & directory, std::string & filename, std::string & buf);
void prepare(size_t s, std::string & filename, std::string & buf);
void prepare2(std::string & filename, std::string & buf);
void prepare3(std::string & filename, std::string & buf);
void prepare4(std::string & filename, std::string & buf);
std::string createTmpFile();
void die(const std::string & msg);
void run_test(unsigned int num, const std::function<bool()> func);
void runTest(unsigned int num, const std::function<bool()> func);
bool test1(const std::string & filename);
bool test2(const std::string & filename, const std::string & buf);
@ -46,30 +47,25 @@ void run()
{
namespace fs = boost::filesystem;
std::string directory;
std::string filename;
std::string buf;
prepare(10 * DEFAULT_AIO_FILE_BLOCK_SIZE, directory, filename, buf);
prepare(10 * DEFAULT_AIO_FILE_BLOCK_SIZE, filename, buf);
std::string directory2;
std::string filename2;
std::string buf2;
prepare(2 * DEFAULT_AIO_FILE_BLOCK_SIZE - 3, directory2, filename2, buf2);
prepare(2 * DEFAULT_AIO_FILE_BLOCK_SIZE - 3, filename2, buf2);
std::string directory3;
std::string filename3;
std::string buf3;
prepare2(directory3, filename3, buf3);
prepare2(filename3, buf3);
std::string directory4;
std::string filename4;
std::string buf4;
prepare3(directory4, filename4, buf4);
prepare3(filename4, buf4);
std::string directory5;
std::string filename5;
std::string buf5;
prepare4(directory5, filename5, buf5);
prepare4(filename5, buf5);
const std::vector<std::function<bool()> > tests =
{
@ -99,27 +95,21 @@ void run()
for (const auto & test : tests)
{
++num;
run_test(num, test);
runTest(num, test);
}
fs::remove_all(directory);
fs::remove_all(directory2);
fs::remove_all(directory3);
fs::remove_all(directory4);
fs::remove_all(directory5);
fs::remove_all(fs::path(filename).parent_path().string());
fs::remove_all(fs::path(filename2).parent_path().string());
fs::remove_all(fs::path(filename3).parent_path().string());
fs::remove_all(fs::path(filename4).parent_path().string());
fs::remove_all(fs::path(filename5).parent_path().string());
}
void prepare(size_t s, std::string & directory, std::string & filename, std::string & buf)
void prepare(size_t s, 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";
filename = createTmpFile();
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
buf.reserve(n);
@ -134,15 +124,9 @@ void prepare(size_t s, std::string & directory, std::string & filename, std::st
out << buf;
}
void prepare2(std::string & directory, std::string & filename, std::string & buf)
void prepare2(std::string & filename, std::string & buf)
{
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
if (dir == nullptr)
die("Could not create directory");
directory = std::string(dir);
filename = directory + "/foo";
filename = createTmpFile();
buf = "122333444455555666666777777788888888999999999";
@ -153,15 +137,9 @@ void prepare2(std::string & directory, std::string & filename, std::string & buf
out << buf;
}
void prepare3(std::string & directory, std::string & filename, std::string & buf)
void prepare3(std::string & filename, std::string & buf)
{
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
if (dir == nullptr)
die("Could not create directory");
directory = std::string(dir);
filename = directory + "/foo";
filename = createTmpFile();
buf = "122333444455555666666777777788888888999999999";
@ -173,17 +151,11 @@ void prepare3(std::string & directory, std::string & filename, std::string & buf
out << buf;
}
void prepare4(std::string & directory, std::string & filename, std::string & buf)
void prepare4(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";
filename = createTmpFile();
std::ofstream out(filename.c_str());
if (!out.is_open())
@ -196,13 +168,23 @@ void prepare4(std::string & directory, std::string & filename, std::string & buf
out << buf;
}
std::string createTmpFile()
{
char pattern[] = "/tmp/fileXXXXXX";
char * dir = ::mkdtemp(pattern);
if (dir == nullptr)
die("Could not create directory");
return std::string(dir) + "/foo";
}
void die(const std::string & msg)
{
std::cout << msg << "\n";
::exit(EXIT_FAILURE);
}
void run_test(unsigned int num, const std::function<bool()> func)
void runTest(unsigned int num, const std::function<bool()> func)
{
bool ok;

View File

@ -11,9 +11,13 @@
namespace
{
namespace fs = boost::filesystem;
void run();
void die(const std::string & msg);
void run_test(unsigned int num, const std::function<bool()> func);
void runTest(unsigned int num, const std::function<bool()> func);
std::string createTmpFile();
std::string generateString(size_t n);
bool test1();
bool test2();
@ -46,7 +50,7 @@ void run()
for (const auto & test : tests)
{
++num;
run_test(num, test);
runTest(num, test);
}
}
@ -56,7 +60,7 @@ void die(const std::string & msg)
::exit(EXIT_FAILURE);
}
void run_test(unsigned int num, const std::function<bool()> func)
void runTest(unsigned int num, const std::function<bool()> func)
{
bool ok;
@ -81,21 +85,19 @@ void run_test(unsigned int num, const std::function<bool()> func)
std::cout << "Test " << num << " failed\n";
}
bool test1()
std::string createTmpFile()
{
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";
return std::string(dir) + "/foo";
}
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
std::string generateString(size_t n)
{
static const std::string symbols = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789";
std::string buf;
buf.reserve(n);
@ -103,6 +105,17 @@ bool test1()
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
return buf;
}
bool test1()
{
std::string filename = createTmpFile();
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE;
std::string buf = generateString(n);
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
@ -121,32 +134,18 @@ bool test1()
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
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";
std::string filename = createTmpFile();
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()];
std::string buf = generateString(n);
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
@ -168,7 +167,7 @@ bool test2()
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
if (received.substr(0, buf.length() / 2) != buf.substr(0, buf.length() / 2))
return false;
@ -182,25 +181,11 @@ bool test2()
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";
std::string filename = createTmpFile();
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()];
std::string buf = generateString(n);
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
@ -229,32 +214,18 @@ bool test3()
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
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";
std::string filename = createTmpFile();
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()];
std::string buf = generateString(n);
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
@ -283,7 +254,7 @@ bool test4()
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
if (received.substr(0, buf.length()) != buf)
return false;
@ -296,25 +267,11 @@ bool test4()
bool test5()
{
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";
std::string filename = createTmpFile();
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()];
std::string buf = generateString(n);
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
@ -335,32 +292,18 @@ bool test5()
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
return received.substr(1) == buf;
}
bool test6()
{
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";
std::string filename = createTmpFile();
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()];
std::string buf = generateString(n);
std::string buf2 = "1111111111";
@ -385,7 +328,7 @@ bool test6()
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
if (received.substr(3, 8 * DEFAULT_AIO_FILE_BLOCK_SIZE) != buf.substr(0, 8 * DEFAULT_AIO_FILE_BLOCK_SIZE))
return false;
@ -401,17 +344,7 @@ bool test6()
bool test7()
{
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";
std::string filename = createTmpFile();
std::string buf2 = "11111111112222222222";
@ -441,24 +374,14 @@ bool test7()
return false;
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
return true;
}
bool test8()
{
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";
std::string filename = createTmpFile();
std::string buf2 = "11111111112222222222";
@ -488,32 +411,18 @@ bool test8()
return false;
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
return true;
}
bool test9()
{
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";
std::string filename = createTmpFile();
size_t n = 3 * DEFAULT_AIO_FILE_BLOCK_SIZE;
std::string buf;
buf.reserve(n);
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
std::string buf = generateString(n);
std::string buf2(DEFAULT_AIO_FILE_BLOCK_SIZE + 10, '1');
@ -538,7 +447,7 @@ bool test9()
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
if (received.substr(3, 2 * DEFAULT_AIO_FILE_BLOCK_SIZE) != buf.substr(0, 2 * DEFAULT_AIO_FILE_BLOCK_SIZE))
return false;
@ -551,25 +460,11 @@ bool test9()
bool test10()
{
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";
std::string filename = createTmpFile();
size_t n = 10 * DEFAULT_AIO_FILE_BLOCK_SIZE + 3;
std::string buf;
buf.reserve(n);
for (size_t i = 0; i < n; ++i)
buf += symbols[i % symbols.length()];
std::string buf = generateString(n);
{
DB::WriteBufferAIO out(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE);
@ -589,7 +484,7 @@ bool test10()
std::string received{ std::istreambuf_iterator<char>(in), std::istreambuf_iterator<char>() };
in.close();
fs::remove_all(directory);
fs::remove_all(fs::path(filename).parent_path().string());
return (received == buf);
}

View File

@ -59,6 +59,9 @@ void AggregatedDataVariants::convertToTwoLevel()
void Aggregator::initialize(Block & block)
{
if (isCancelled())
return;
std::lock_guard<std::mutex> lock(mutex);
if (initialized)
@ -84,6 +87,9 @@ void Aggregator::initialize(Block & block)
all_aggregates_has_trivial_destructor = false;
}
if (isCancelled())
return;
/** Всё остальное - только если передан непустой block.
* (всё остальное не нужно в методе merge блоков с готовыми состояниями агрегатных функций).
*/
@ -100,6 +106,9 @@ void Aggregator::initialize(Block & block)
for (Names::const_iterator jt = it->argument_names.begin(); jt != it->argument_names.end(); ++jt)
it->arguments.push_back(block.getPositionByName(*jt));
if (isCancelled())
return;
/// Создадим пример блока, описывающего результат
if (!sample)
{
@ -539,6 +548,9 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
{
initialize(block);
if (isCancelled())
return true;
/// result будет уничтожать состояния агрегатных функций в деструкторе
result.aggregator = this;
@ -569,6 +581,9 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
}
}
if (isCancelled())
return true;
size_t rows = block.rows();
/// Каким способом выполнять агрегацию?
@ -583,6 +598,9 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
compileIfPossible(result.type);
}
if (isCancelled())
return true;
if ((overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key)
{
AggregateDataPtr place = result.aggregates_pool->alloc(total_size_of_aggregate_states);
@ -686,6 +704,9 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
*/
void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & result)
{
if (isCancelled())
return;
StringRefs key(keys_size);
ConstColumnPlainPtrs key_columns(keys_size);
AggregateColumns aggregate_columns(aggregates_size);
@ -708,6 +729,9 @@ void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & re
/// Читаем все данные
while (Block block = stream->read())
{
if (isCancelled())
return;
src_rows += block.rows();
src_bytes += block.bytes();
@ -1053,6 +1077,9 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, bool final, size_t max_threads)
{
if (isCancelled())
return BlocksList();
LOG_TRACE(log, "Converting aggregated data to blocks");
Stopwatch watch;
@ -1070,9 +1097,15 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
try
{
if (isCancelled())
return BlocksList();
if (data_variants.type == AggregatedDataVariants::Type::without_key || overflow_row)
blocks.splice(blocks.end(), prepareBlocksAndFillWithoutKey(data_variants, final));
if (isCancelled())
return BlocksList();
if (data_variants.type != AggregatedDataVariants::Type::without_key)
{
if (!data_variants.isTwoLevel())
@ -1105,6 +1138,9 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
data_variants.aggregator = nullptr;
}
if (isCancelled())
return BlocksList();
size_t rows = 0;
size_t bytes = 0;
@ -1463,6 +1499,9 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads)
{
if (isCancelled())
return;
StringRefs key(keys_size);
ConstColumnPlainPtrs key_columns(keys_size);
@ -1471,6 +1510,9 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
Block empty_block;
initialize(empty_block);
if (isCancelled())
return;
/** Если на удалённых серверах использовался двухуровневый метод агрегации,
* то в блоках будет расположена информация о номере корзины.
* Тогда вычисления можно будет распараллелить по корзинам.
@ -1486,6 +1528,9 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
size_t total_input_blocks = 0;
while (Block block = stream->read())
{
if (isCancelled())
return;
total_input_rows += block.rowsInFirstColumn();
++total_input_blocks;
bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block));
@ -1524,6 +1569,9 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
#undef M
}
if (isCancelled())
return;
/// result будет уничтожать состояния агрегатных функций в деструкторе
result.aggregator = this;
@ -1544,6 +1592,9 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
for (Block & block : bucket_to_blocks[bucket])
{
if (isCancelled())
return;
#define M(NAME) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
mergeStreamsImpl(block, key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket]);
@ -1593,6 +1644,12 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
LOG_TRACE(log, "Merged partially aggregated two-level data.");
}
if (isCancelled())
{
result.invalidate();
return;
}
if (has_blocks_with_unknown_bucket)
{
LOG_TRACE(log, "Merging partially aggregated single-level data.");
@ -1600,6 +1657,12 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
BlocksList & blocks = bucket_to_blocks[-1];
for (Block & block : blocks)
{
if (isCancelled())
{
result.invalidate();
return;
}
if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows)
mergeWithoutKeyStreamsImpl(block, result);
@ -1694,4 +1757,9 @@ String Aggregator::getID() const
return res.str();
}
void Aggregator::setCancellationHook(const CancellationHook cancellation_hook)
{
isCancelled = cancellation_hook;
}
}

View File

@ -1,20 +1,171 @@
#include <map>
#include <set>
#include <Poco/SharedPtr.h>
#include <Poco/Mutex.h>
#include <Poco/File.h>
#include <Poco/Net/NetworkInterface.h>
#include <Yandex/logger_useful.h>
#include <DB/Common/Macros.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/TableFunctions/TableFunctionFactory.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/Storages/IStorage.h>
#include <DB/Storages/MarkCache.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
#include <DB/Storages/MergeTree/MergeList.h>
#include <DB/Storages/CompressionMethodSelector.h>
#include <DB/Interpreters/Settings.h>
#include <DB/Interpreters/Users.h>
#include <DB/Interpreters/Quota.h>
#include <DB/Interpreters/Dictionaries.h>
#include <DB/Interpreters/ExternalDictionaries.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Interpreters/Cluster.h>
#include <DB/Interpreters/InterserverIOHandler.h>
#include <DB/Interpreters/Compiler.h>
#include <DB/Interpreters/Context.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/copyData.h>
#include <DB/IO/UncompressedCache.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Interpreters/Context.h>
#include <DB/Client/ConnectionPool.h>
#include <DB/Client/ConnectionPoolWithFailover.h>
#include <statdaemons/ConfigProcessor.h>
#include <zkutil/ZooKeeper.h>
namespace DB
{
class TableFunctionFactory;
using Poco::SharedPtr;
/** Набор известных объектов, которые могут быть использованы в запросе.
* Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен.
*/
struct ContextShared
{
Logger * log = &Logger::get("Context"); /// Логгер.
mutable Poco::Mutex mutex; /// Для доступа и модификации разделяемых объектов.
mutable zkutil::ZooKeeperPtr zookeeper; /// Клиент для ZooKeeper.
String interserver_io_host; /// Имя хоста по которым это сервер доступен для других серверов.
int interserver_io_port; /// и порт,
String path; /// Путь к директории с данными, со слешем на конце.
String tmp_path; /// Путь ко временным файлам, возникающим при обработке запроса.
Databases databases; /// Список БД и таблиц в них.
TableFunctionFactory table_function_factory; /// Табличные функции.
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
DataTypeFactory data_type_factory; /// Типы данных.
FormatFactory format_factory; /// Форматы.
mutable SharedPtr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво.
mutable SharedPtr<ExternalDictionaries> external_dictionaries;
Users users; /// Известные пользователи.
Quotas quotas; /// Известные квоты на использование ресурсов.
mutable UncompressedCachePtr uncompressed_cache; /// Кэш разжатых блоков.
mutable MarkCachePtr mark_cache; /// Кэш засечек в сжатых файлах.
ProcessList process_list; /// Исполняющиеся в данный момент запросы.
MergeList merge_list; /// Список выполняемых мерджей (для (Replicated)?MergeTree)
ViewDependencies view_dependencies; /// Текущие зависимости
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
InterserverIOHandler interserver_io_handler; /// Обработчик для межсерверной передачи данных.
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
Macros macros; /// Подстановки из конфига.
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска.
/// Кластеры для distributed таблиц
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
Poco::SharedPtr<Clusters> clusters;
bool shutdown_called = false;
~ContextShared()
{
try
{
shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
/** Выполнить сложную работу по уничтожению объектов заранее.
*/
void shutdown()
{
if (shutdown_called)
return;
shutdown_called = true;
/** В этот момент, некоторые таблицы могут иметь потоки,
* которые модифицируют список таблиц, и блокируют наш mutex (см. StorageChunkMerger).
* Чтобы корректно их завершить, скопируем текущий список таблиц,
* и попросим их всех закончить свою работу.
* Потом удалим все объекты с таблицами.
*/
Databases current_databases;
{
Poco::ScopedLock<Poco::Mutex> lock(mutex);
current_databases = databases;
}
for (Databases::iterator it = current_databases.begin(); it != current_databases.end(); ++it)
for (Tables::iterator jt = it->second.begin(); jt != it->second.end(); ++jt)
jt->second->shutdown();
{
Poco::ScopedLock<Poco::Mutex> lock(mutex);
databases.clear();
}
}
};
Context::Context()
: shared(new ContextShared),
quota(new QuotaForIntervals)
{
}
Context::~Context() = default;
const TableFunctionFactory & Context::getTableFunctionFactory() const { return shared->table_function_factory; }
const AggregateFunctionFactory & Context::getAggregateFunctionFactory() const { return shared->aggregate_function_factory; }
const DataTypeFactory & Context::getDataTypeFactory() const { return shared->data_type_factory; }
const FormatFactory & Context::getFormatFactory() const { return shared->format_factory; }
InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; }
Poco::Mutex & Context::getMutex() const { return shared->mutex; }
const Databases & Context::getDatabases() const { return shared->databases; }
Databases & Context::getDatabases() { return shared->databases; }
ProcessList & Context::getProcessList() { return shared->process_list; }
const ProcessList & Context::getProcessList() const { return shared->process_list; }
MergeList & Context::getMergeList() { return shared->merge_list; }
const MergeList & Context::getMergeList() const { return shared->merge_list; }
String Context::getPath() const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
@ -561,13 +712,13 @@ void Context::setUncompressedCache(size_t max_size_in_bytes)
if (shared->uncompressed_cache)
throw Exception("Uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->uncompressed_cache = new UncompressedCache(max_size_in_bytes);
shared->uncompressed_cache.reset(new UncompressedCache(max_size_in_bytes));
}
UncompressedCachePtr Context::getUncompressedCache() const
{
/// Исходим из допущения, что функция setUncompressedCache, если вызывалась, то раньше. Иначе поставьте mutex.
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return shared->uncompressed_cache;
}
@ -578,12 +729,12 @@ void Context::setMarkCache(size_t cache_size_in_bytes)
if (shared->mark_cache)
throw Exception("Uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->mark_cache = new MarkCache(cache_size_in_bytes);
shared->mark_cache.reset(new MarkCache(cache_size_in_bytes));
}
MarkCachePtr Context::getMarkCache() const
{
/// Исходим из допущения, что функция setMarksCache, если вызывалась, то раньше. Иначе поставьте mutex.
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return shared->mark_cache;
}
@ -597,7 +748,7 @@ BackgroundProcessingPool & Context::getBackgroundPool()
void Context::resetCaches() const
{
/// Исходим из допущения, что функции setUncompressedCache, setMarkCache, если вызывались, то раньше (при старте сервера). Иначе поставьте mutex.
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (shared->uncompressed_cache)
shared->uncompressed_cache->reset();
@ -675,4 +826,28 @@ Compiler & Context::getCompiler()
}
CompressionMethod Context::chooseCompressionMethod(size_t part_size, double part_size_ratio) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (!shared->compression_method_selector)
{
constexpr auto config_name = "compression";
auto & config = Poco::Util::Application::instance().config();
if (config.has(config_name))
shared->compression_method_selector.reset(new CompressionMethodSelector{config, "compression"});
else
shared->compression_method_selector.reset(new CompressionMethodSelector);
}
return shared->compression_method_selector->choose(part_size, part_size_ratio);
}
void Context::shutdown()
{
shared->shutdown();
}
}

View File

@ -215,7 +215,7 @@ void ExpressionAction::prepare(Block & sample_block)
void ExpressionAction::execute(Block & block) const
{
// std::cerr << "executing: " << toString() << std::endl;
std::cerr << "executing: " << toString() << std::endl;
if (type == REMOVE_COLUMN || type == COPY_COLUMN)
if (!block.has(source_name))
@ -339,6 +339,16 @@ void ExpressionAction::execute(Block & block) const
}
}
void ExpressionAction::executeOnTotals(Block & block) const
{
if (type != JOIN)
execute(block);
else
join->joinTotals(block);
}
std::string ExpressionAction::toString() const
{
std::stringstream ss;
@ -543,6 +553,38 @@ void ExpressionActions::execute(Block & block) const
}
}
void ExpressionActions::executeOnTotals(Block & block) const
{
/// Если в подзапросе для JOIN-а есть totals, а у нас нет, то возьмём блок со значениями по-умолчанию вместо totals.
if (!block)
{
bool has_totals_in_join = false;
for (const auto & action : actions)
{
if (action.join && action.join->hasTotals())
{
has_totals_in_join = true;
break;
}
}
if (has_totals_in_join)
{
for (const auto & name_and_type : input_columns)
{
ColumnWithNameAndType elem(name_and_type.type->createColumn(), name_and_type.type, name_and_type.name);
elem.column->insertDefault();
block.insert(elem);
}
}
else
return; /// Нечего JOIN-ить.
}
for (const auto & action : actions)
action.executeOnTotals(block);
}
std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & columns)
{
NamesAndTypesList::const_iterator it = columns.begin();
@ -833,6 +875,24 @@ void ExpressionActions::optimizeArrayJoin()
}
BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRightJoin(size_t max_block_size) const
{
for (const auto & action : actions)
{
if (action.join && (action.join->getKind() == ASTJoin::Full || action.join->getKind() == ASTJoin::Right))
{
Block left_sample_block;
for (const auto & input_elem : input_columns)
left_sample_block.insert(ColumnWithNameAndType(nullptr, input_elem.type, input_elem.name));
return action.join->createStreamWithNonJoinedRows(left_sample_block, max_block_size);
}
}
return {};
}
void ExpressionActionsChain::addStep()
{
if (steps.empty())

View File

@ -9,7 +9,6 @@
#include <DB/Parsers/ASTSubquery.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Parsers/ASTOrderByElement.h>
#include <DB/Parsers/ParserSelectQuery.h>
#include <DB/DataTypes/DataTypeSet.h>
#include <DB/DataTypes/DataTypeTuple.h>
@ -23,6 +22,8 @@
#include <DB/Interpreters/ExpressionAnalyzer.h>
#include <DB/Interpreters/LogicalExpressionsOptimizer.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/Storages/StorageDistributed.h>
#include <DB/Storages/StorageMemory.h>
#include <DB/Storages/StorageSet.h>
@ -984,7 +985,7 @@ struct ExpressionAnalyzer::ScopeStack
return res;
}
const Block & getSampleBlock()
const Block & getSampleBlock() const
{
return stack.back().actions->getSampleBlock();
}
@ -1395,19 +1396,19 @@ void ExpressionAnalyzer::getAggregates(ASTPtr ast, ExpressionActionsPtr & action
}
}
void ExpressionAnalyzer::assertSelect()
void ExpressionAnalyzer::assertSelect() const
{
if (!select_query)
throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR);
}
void ExpressionAnalyzer::assertAggregation()
void ExpressionAnalyzer::assertAggregation() const
{
if (!has_aggregation)
throw Exception("No aggregation", ErrorCodes::LOGICAL_ERROR);
}
void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, NamesAndTypesList & columns)
void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const
{
if (chain.steps.empty())
{
@ -1416,14 +1417,14 @@ void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, NamesAndTypes
}
}
void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions)
void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const
{
NameSet result_columns;
for (NameToNameMap::iterator it = array_join_result_to_source.begin(); it != array_join_result_to_source.end(); ++it)
for (const auto & result_source : array_join_result_to_source)
{
if (it->first != it->second)
actions->add(ExpressionAction::copyColumn(it->second, it->first));
result_columns.insert(it->first);
if (result_source.first != result_source.second)
actions->add(ExpressionAction::copyColumn(result_source.second, result_source.first));
result_columns.insert(result_source.first);
}
actions->add(ExpressionAction::arrayJoin(result_columns));
@ -1446,7 +1447,7 @@ bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool on
return true;
}
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only_types)
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only_types) const
{
if (only_types)
actions->add(ExpressionAction::ordinaryJoin(nullptr, columns_added_by_join));
@ -1642,7 +1643,7 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only
return true;
}
void ExpressionAnalyzer::appendProjectResult(DB::ExpressionActionsChain & chain, bool only_types)
void ExpressionAnalyzer::appendProjectResult(DB::ExpressionActionsChain & chain, bool only_types) const
{
assertSelect();
@ -1702,7 +1703,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result)
ASTs asts;
if (ASTExpressionList * node = typeid_cast<ASTExpressionList *>(&*ast))
if (auto node = typeid_cast<const ASTExpressionList *>(&*ast))
asts = node->children;
else
asts = ASTs(1, ast);
@ -1746,10 +1747,10 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
return actions;
}
void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates)
void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const
{
for (NamesAndTypesList::iterator it = aggregation_keys.begin(); it != aggregation_keys.end(); ++it)
key_names.emplace_back(it->name);
for (const auto & name_and_type : aggregation_keys)
key_names.emplace_back(name_and_type.name);
aggregates = aggregate_descriptions;
}

View File

@ -9,6 +9,7 @@
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/IO/copyData.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/parseQuery.h>
@ -41,7 +42,7 @@ void InterpreterAlterQuery::execute()
switch (command.type)
{
case PartitionCommand::DROP_PARTITION:
table->dropPartition(command.partition, command.detach, context.getSettingsRef());
table->dropPartition(command.partition, command.detach, command.unreplicated, context.getSettingsRef());
break;
case PartitionCommand::ATTACH_PARTITION:
@ -134,7 +135,7 @@ void InterpreterAlterQuery::parseAlter(
else if (params.type == ASTAlterQuery::DROP_PARTITION)
{
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
out_partition_commands.push_back(PartitionCommand::dropPartition(partition, params.detach));
out_partition_commands.push_back(PartitionCommand::dropPartition(partition, params.detach, params.unreplicated));
}
else if (params.type == ASTAlterQuery::ATTACH_PARTITION)
{

View File

@ -17,8 +17,10 @@
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/InterpreterCreateQuery.h>
@ -409,11 +411,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
const auto end = pos + type_name->size();
ParserIdentifierWithOptionalParameters storage_p;
Expected expected{""};
IParser::Pos max_parsed_pos = pos;
if (!storage_p.parse(pos, end, column_declaration->type, max_parsed_pos, expected))
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type->query_string = type_name;
columns_list.children.push_back(column_declaration_ptr);
}
@ -444,11 +442,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(NamesAndTypesList columns,
const auto end = pos + type_name->size();
ParserIdentifierWithOptionalParameters storage_p;
Expected expected{""};
IParser::Pos max_parsed_pos = pos;
if (!storage_p.parse(pos, end, column_declaration->type, max_parsed_pos, expected))
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type->query_string = type_name;
const auto it = column_defaults.find(column.name);

View File

@ -5,6 +5,7 @@
#include <DB/DataStreams/AddingDefaultBlockOutputStream.h>
#include <DB/DataStreams/PushingToViewsBlockOutputStream.h>
#include <DB/DataStreams/NullAndDoCopyBlockInputStream.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/DataStreams/copyData.h>
#include <DB/Parsers/ASTInsertQuery.h>
@ -114,8 +115,8 @@ void InterpreterInsertQuery::execute(ReadBuffer * remaining_data_istr)
BlockInputStreamPtr in{
context.getFormatFactory().getInput(
format, istr, sample, context.getSettings().max_insert_block_size,
context.getDataTypeFactory())
};
context.getDataTypeFactory())};
copyData(*in, *out);
}
else

View File

@ -15,6 +15,7 @@
#include <DB/DataStreams/copyData.h>
#include <DB/DataStreams/CreatingSetsBlockInputStream.h>
#include <DB/DataStreams/MaterializingBlockInputStream.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTIdentifier.h>
@ -292,7 +293,7 @@ Block InterpreterSelectQuery::getSampleBlock()
BlockInputStreamPtr InterpreterSelectQuery::execute()
{
(void) executeWithoutUnion();
(void) executeWithoutUnion(false);
if (streams.empty())
return new NullBlockInputStream;
@ -322,8 +323,10 @@ BlockInputStreamPtr InterpreterSelectQuery::execute()
return streams[0];
}
const BlockInputStreams & InterpreterSelectQuery::executeWithoutUnion()
const BlockInputStreams & InterpreterSelectQuery::executeWithoutUnion(bool force_no_union_)
{
force_no_union = force_no_union_;
if (is_first_select_inside_union_all)
{
executeSingleQuery();
@ -558,7 +561,7 @@ void InterpreterSelectQuery::executeSingleQuery()
if (need_second_distinct_pass)
do_execute_union = true;
if (do_execute_union)
if (do_execute_union && !force_no_union)
executeUnion(streams);
/// Если было более одного источника - то нужно выполнить DISTINCT ещё раз после их слияния.

View File

@ -3,6 +3,7 @@
#include <DB/Parsers/ASTJoin.h>
#include <DB/Interpreters/Join.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
@ -89,34 +90,52 @@ static size_t getTotalByteCountImpl(const Maps & maps)
}
/// Нужно ли использовать хэш-таблицы maps_*_full, в которых запоминается, была ли строчка присоединена.
static bool getFullness(ASTJoin::Kind kind)
{
return kind == ASTJoin::Right || kind == ASTJoin::Full;
}
void Join::init(Type type_)
{
type = type_;
if (strictness == ASTJoin::Any)
initImpl(maps_any, type);
if (!getFullness(kind))
{
if (strictness == ASTJoin::Any)
initImpl(maps_any, type);
else
initImpl(maps_all, type);
}
else
initImpl(maps_all, type);
{
if (strictness == ASTJoin::Any)
initImpl(maps_any_full, type);
else
initImpl(maps_all_full, type);
}
}
size_t Join::getTotalRowCount() const
{
if (strictness == ASTJoin::Any)
return getTotalRowCountImpl(maps_any);
else
return getTotalRowCountImpl(maps_all);
size_t res = 0;
res += getTotalRowCountImpl(maps_any);
res += getTotalRowCountImpl(maps_all);
res += getTotalRowCountImpl(maps_any_full);
res += getTotalRowCountImpl(maps_all_full);
return res;
}
size_t Join::getTotalByteCount() const
{
size_t bytes;
if (strictness == ASTJoin::Any)
bytes = getTotalByteCountImpl(maps_any);
else
bytes = getTotalByteCountImpl(maps_all);
return bytes + pool.size();
size_t res = 0;
res += getTotalByteCountImpl(maps_any);
res += getTotalByteCountImpl(maps_all);
res += getTotalByteCountImpl(maps_any_full);
res += getTotalByteCountImpl(maps_all_full);
res += pool.size();
return res;
}
@ -147,28 +166,32 @@ struct Inserter<ASTJoin::Any, Map>
map.emplace(key, it, inserted);
if (inserted)
new (&it->second) Join::RowRef(stored_block, i);
new (&it->second) typename Map::mapped_type(stored_block, i);
}
};
/// Для строковых ключей отличается тем, что саму строчку надо разместить в пуле.
template <>
struct Inserter<ASTJoin::Any, Join::MapsAny::MapString>
template <typename Map>
struct InserterAnyString
{
static void insert(Join::MapsAny::MapString & map, const Join::MapsAny::MapString::key_type & key, Block * stored_block, size_t i, Arena & pool)
static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool)
{
Join::MapsAny::MapString::iterator it;
typename Map::iterator it;
bool inserted;
map.emplace(key, it, inserted);
if (inserted)
{
it->first.data = pool.insert(key.data, key.size);
new (&it->second) Join::RowRef(stored_block, i);
new (&it->second) typename Map::mapped_type(stored_block, i);
}
}
};
template <> struct Inserter<ASTJoin::Any, Join::MapsAny::MapString> : InserterAnyString<Join::MapsAny::MapString> {};
template <> struct Inserter<ASTJoin::Any, Join::MapsAnyFull::MapString> : InserterAnyString<Join::MapsAnyFull::MapString> {};
template <typename Map>
struct Inserter<ASTJoin::All, Map>
{
@ -180,7 +203,7 @@ struct Inserter<ASTJoin::All, Map>
if (inserted)
{
new (&it->second) Join::RowRefList(stored_block, i);
new (&it->second) typename Map::mapped_type(stored_block, i);
}
else
{
@ -188,7 +211,7 @@ struct Inserter<ASTJoin::All, Map>
* Мы будем вставлять каждый раз элемент на место второго.
* То есть, бывший второй элемент, если он был, станет третьим, и т. п.
*/
Join::RowRefList * elem = reinterpret_cast<Join::RowRefList *>(pool.alloc(sizeof(Join::RowRefList)));
auto elem = reinterpret_cast<typename Map::mapped_type *>(pool.alloc(sizeof(typename Map::mapped_type)));
elem->next = it->second.next;
it->second.next = elem;
@ -198,23 +221,23 @@ struct Inserter<ASTJoin::All, Map>
}
};
template <>
struct Inserter<ASTJoin::All, Join::MapsAll::MapString>
template <typename Map>
struct InserterAllString
{
static void insert(Join::MapsAll::MapString & map, const Join::MapsAll::MapString::key_type & key, Block * stored_block, size_t i, Arena & pool)
static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool)
{
typename Join::MapsAll::MapString::iterator it;
typename Map::iterator it;
bool inserted;
map.emplace(key, it, inserted);
if (inserted)
{
it->first.data = pool.insert(key.data, key.size);
new (&it->second) Join::RowRefList(stored_block, i);
new (&it->second) typename Map::mapped_type(stored_block, i);
}
else
{
Join::RowRefList * elem = reinterpret_cast<Join::RowRefList *>(pool.alloc(sizeof(Join::RowRefList)));
auto elem = reinterpret_cast<typename Map::mapped_type *>(pool.alloc(sizeof(typename Map::mapped_type)));
elem->next = it->second.next;
it->second.next = elem;
@ -224,6 +247,9 @@ struct Inserter<ASTJoin::All, Join::MapsAll::MapString>
}
};
template <> struct Inserter<ASTJoin::All, Join::MapsAll::MapString> : InserterAllString<Join::MapsAll::MapString> {};
template <> struct Inserter<ASTJoin::All, Join::MapsAllFull::MapString> : InserterAllString<Join::MapsAllFull::MapString> {};
template <ASTJoin::Strictness STRICTNESS, typename Maps>
void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block)
@ -321,10 +347,20 @@ bool Join::insertFromBlock(const Block & block)
for (const auto & name : key_names_right)
stored_block->erase(stored_block->getPositionByName(name));
if (strictness == ASTJoin::Any)
insertFromBlockImpl<ASTJoin::Any, MapsAny>(maps_any, rows, key_columns, keys_size, stored_block);
if (!getFullness(kind))
{
if (strictness == ASTJoin::Any)
insertFromBlockImpl<ASTJoin::Any>(maps_any, rows, key_columns, keys_size, stored_block);
else
insertFromBlockImpl<ASTJoin::All>(maps_all, rows, key_columns, keys_size, stored_block);
}
else
insertFromBlockImpl<ASTJoin::All, MapsAll>(maps_all, rows, key_columns, keys_size, stored_block);
{
if (strictness == ASTJoin::Any)
insertFromBlockImpl<ASTJoin::Any>(maps_any_full, rows, key_columns, keys_size, stored_block);
else
insertFromBlockImpl<ASTJoin::All>(maps_all_full, rows, key_columns, keys_size, stored_block);
}
if (!checkSizeLimits())
{
@ -347,11 +383,7 @@ bool Join::insertFromBlock(const Block & block)
template <ASTJoin::Kind KIND, ASTJoin::Strictness STRICTNESS, typename Map>
struct Adder
{
static void add(const Map & map, const typename Map::key_type & key, size_t num_columns_to_add, ColumnPlainPtrs & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset_t & current_offset, IColumn::Offsets_t * offsets);
};
struct Adder;
template <typename Map>
struct Adder<ASTJoin::Left, ASTJoin::Any, Map>
@ -363,6 +395,7 @@ struct Adder<ASTJoin::Left, ASTJoin::Any, Map>
if (it != map.end())
{
it->second.setUsed();
for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertFrom(*it->second.block->unsafeGetByPosition(j).column.get(), it->second.row_num);
}
@ -386,6 +419,7 @@ struct Adder<ASTJoin::Inner, ASTJoin::Any, Map>
{
(*filter)[i] = 1;
it->second.setUsed();
for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertFrom(*it->second.block->unsafeGetByPosition(j).column.get(), it->second.row_num);
}
@ -405,7 +439,8 @@ struct Adder<KIND, ASTJoin::All, Map>
if (it != map.end())
{
size_t rows_joined = 0;
for (const Join::RowRefList * current = &it->second; current != nullptr; current = current->next)
it->second.setUsed();
for (auto current = &static_cast<const typename Map::mapped_type::Base_t &>(it->second); current != nullptr; current = current->next)
{
for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertFrom(*current->block->unsafeGetByPosition(j).column.get(), current->row_num);
@ -469,7 +504,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
/// Используется при ANY INNER JOIN
std::unique_ptr<IColumn::Filter> filter;
if (kind == ASTJoin::Inner && strictness == ASTJoin::Any)
if ((kind == ASTJoin::Inner || kind == ASTJoin::Right) && strictness == ASTJoin::Any)
filter.reset(new IColumn::Filter(rows));
/// Используется при ALL ... JOIN
@ -546,13 +581,13 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
else
throw Exception("Unknown JOIN variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
/// Если ANY INNER JOIN - фильтруем все столбцы кроме новых.
if (kind == ASTJoin::Inner && strictness == ASTJoin::Any)
/// Если ANY INNER|RIGHT JOIN - фильтруем все столбцы кроме новых.
if (filter)
for (size_t i = 0; i < existing_columns; ++i)
block.getByPosition(i).column = block.getByPosition(i).column->filter(*filter);
/// Если ALL ... JOIN - размножаем все столбцы кроме новых.
if (strictness == ASTJoin::All)
if (offsets_to_replicate)
for (size_t i = 0; i < existing_columns; ++i)
block.getByPosition(i).column = block.getByPosition(i).column->replicate(*offsets_to_replicate);
}
@ -563,13 +598,218 @@ void Join::joinBlock(Block & block) const
Poco::ScopedReadRWLock lock(rwlock);
if (kind == ASTJoin::Left && strictness == ASTJoin::Any)
joinBlockImpl<ASTJoin::Left, ASTJoin::Any, MapsAny>(block, maps_any);
joinBlockImpl<ASTJoin::Left, ASTJoin::Any>(block, maps_any);
else if (kind == ASTJoin::Inner && strictness == ASTJoin::Any)
joinBlockImpl<ASTJoin::Inner, ASTJoin::Any, MapsAny>(block, maps_any);
joinBlockImpl<ASTJoin::Inner, ASTJoin::Any>(block, maps_any);
else if (kind == ASTJoin::Left && strictness == ASTJoin::All)
joinBlockImpl<ASTJoin::Left, ASTJoin::All, MapsAll>(block, maps_all);
joinBlockImpl<ASTJoin::Left, ASTJoin::All>(block, maps_all);
else if (kind == ASTJoin::Inner && strictness == ASTJoin::All)
joinBlockImpl<ASTJoin::Inner, ASTJoin::All, MapsAll>(block, maps_all);
joinBlockImpl<ASTJoin::Inner, ASTJoin::All>(block, maps_all);
else if (kind == ASTJoin::Full && strictness == ASTJoin::Any)
joinBlockImpl<ASTJoin::Left, ASTJoin::Any>(block, maps_any_full);
else if (kind == ASTJoin::Right && strictness == ASTJoin::Any)
joinBlockImpl<ASTJoin::Inner, ASTJoin::Any>(block, maps_any_full);
else if (kind == ASTJoin::Full && strictness == ASTJoin::All)
joinBlockImpl<ASTJoin::Left, ASTJoin::All>(block, maps_all_full);
else if (kind == ASTJoin::Right && strictness == ASTJoin::All)
joinBlockImpl<ASTJoin::Inner, ASTJoin::All>(block, maps_all_full);
}
void Join::joinTotals(Block & block) const
{
Block totals_without_keys = totals;
if (totals_without_keys)
{
for (const auto & name : key_names_right)
totals_without_keys.erase(totals_without_keys.getPositionByName(name));
for (size_t i = 0; i < totals_without_keys.columns(); ++i)
block.insert(totals_without_keys.getByPosition(i));
}
else
{
if (blocks.empty())
return;
/// Будем присоединять пустые totals - из одной строчки со значениями по-умолчанию.
totals_without_keys = blocks.front().cloneEmpty();
for (size_t i = 0; i < totals_without_keys.columns(); ++i)
{
totals_without_keys.getByPosition(i).column->insertDefault();
block.insert(totals_without_keys.getByPosition(i));
}
}
}
template <ASTJoin::Strictness STRICTNESS, typename Mapped>
struct AdderNonJoined;
template <typename Mapped>
struct AdderNonJoined<ASTJoin::Any, Mapped>
{
static void add(const Mapped & mapped,
size_t num_columns_left, ColumnPlainPtrs & columns_left,
size_t num_columns_right, ColumnPlainPtrs & columns_right)
{
for (size_t j = 0; j < num_columns_left; ++j)
columns_left[j]->insertDefault();
for (size_t j = 0; j < num_columns_right; ++j)
columns_right[j]->insertFrom(*mapped.block->unsafeGetByPosition(j).column.get(), mapped.row_num);
}
};
template <typename Mapped>
struct AdderNonJoined<ASTJoin::All, Mapped>
{
static void add(const Mapped & mapped,
size_t num_columns_left, ColumnPlainPtrs & columns_left,
size_t num_columns_right, ColumnPlainPtrs & columns_right)
{
for (auto current = &static_cast<const typename Mapped::Base_t &>(mapped); current != nullptr; current = current->next)
{
for (size_t j = 0; j < num_columns_left; ++j)
columns_left[j]->insertDefault();
for (size_t j = 0; j < num_columns_right; ++j)
columns_right[j]->insertFrom(*current->block->unsafeGetByPosition(j).column.get(), current->row_num);
}
}
};
/// Поток из неприсоединённых ранее строк правой таблицы.
class NonJoinedBlockInputStream : public IProfilingBlockInputStream
{
public:
NonJoinedBlockInputStream(const Join & parent_, Block & left_sample_block_, size_t max_block_size_)
: parent(parent_), left_sample_block(left_sample_block_), max_block_size(max_block_size_)
{
}
String getName() const override { return "NonJoinedBlockInputStream"; }
String getID() const override
{
std::stringstream res;
res << "NonJoined(" << &parent << ")";
return res.str();
}
protected:
Block readImpl() override
{
if (parent.blocks.empty())
return Block();
if (parent.strictness == ASTJoin::Any)
return createBlock<ASTJoin::Any>(parent.maps_any_full);
else if (parent.strictness == ASTJoin::All)
return createBlock<ASTJoin::All>(parent.maps_all_full);
else
throw Exception("Logical error: unknown JOIN strictness (must be ANY or ALL)", ErrorCodes::LOGICAL_ERROR);
}
private:
const Join & parent;
Block left_sample_block;
size_t max_block_size;
std::unique_ptr<void, std::function<void(void *)>> position; /// type erasure
template <ASTJoin::Strictness STRICTNESS, typename Maps>
Block createBlock(const Maps & maps)
{
Block block = left_sample_block.cloneEmpty();
size_t num_columns_left = left_sample_block.columns();
ColumnPlainPtrs columns_left(num_columns_left);
for (size_t i = 0; i < num_columns_left; ++i)
{
auto & column_with_name_and_type = block.getByPosition(i);
column_with_name_and_type.column = column_with_name_and_type.type->createColumn();
columns_left[i] = column_with_name_and_type.column.get();
}
/// Добавляем в блок новые столбцы.
const Block & first_mapped_block = parent.blocks.front();
size_t num_columns_right = first_mapped_block.columns();
ColumnPlainPtrs columns_right(num_columns_right);
for (size_t i = 0; i < num_columns_right; ++i)
{
const ColumnWithNameAndType & src_column = first_mapped_block.getByPosition(i);
ColumnWithNameAndType new_column = src_column.cloneEmpty();
block.insert(new_column);
columns_right[i] = new_column.column;
columns_right[i]->reserve(src_column.column->size());
}
size_t rows_added = 0;
if (parent.type == Join::Type::KEY_64)
rows_added = fillColumns<STRICTNESS>(*maps.key64, num_columns_left, columns_left, num_columns_right, columns_right);
else if (parent.type == Join::Type::KEY_STRING)
rows_added = fillColumns<STRICTNESS>(*maps.key_string, num_columns_left, columns_left, num_columns_right, columns_right);
else if (parent.type == Join::Type::HASHED)
rows_added = fillColumns<STRICTNESS>(*maps.hashed, num_columns_left, columns_left, num_columns_right, columns_right);
else
throw Exception("Unknown JOIN variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
std::cerr << "rows added: " << rows_added << "\n";
if (!rows_added)
return Block();
std::cerr << block.dumpStructure() << "\n";
return block;
}
template <ASTJoin::Strictness STRICTNESS, typename Map>
size_t fillColumns(const Map & map,
size_t num_columns_left, ColumnPlainPtrs & columns_left,
size_t num_columns_right, ColumnPlainPtrs & columns_right)
{
size_t rows_added = 0;
if (!position)
position = decltype(position)(
static_cast<void *>(new typename Map::const_iterator(map.begin())),
[](void * ptr) { delete reinterpret_cast<typename Map::const_iterator *>(ptr); });
auto & it = *reinterpret_cast<typename Map::const_iterator *>(position.get());
auto end = map.end();
for (; it != end; ++it)
{
std::cerr << it->second.getUsed() << "\n";
if (it->second.getUsed())
continue;
AdderNonJoined<STRICTNESS, typename Map::mapped_type>::add(it->second, num_columns_left, columns_left, num_columns_right, columns_right);
++rows_added;
if (rows_added == max_block_size)
break;
}
return rows_added;
}
};
BlockInputStreamPtr Join::createStreamWithNonJoinedRows(Block & left_sample_block, size_t max_block_size) const
{
return new NonJoinedBlockInputStream(*this, left_sample_block, max_block_size);
}

View File

@ -260,7 +260,7 @@ QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_na
Container::iterator it = quota_for_keys.find(quota_key_hashed);
if (quota_for_keys.end() == it)
{
it = quota_for_keys.insert(std::make_pair(quota_key_hashed, new QuotaForIntervals(max))).first;
it = quota_for_keys.insert(std::make_pair(quota_key_hashed, std::make_shared<QuotaForIntervals>(max))).first;
}
return it->second;

View File

@ -5,7 +5,9 @@
#include <DB/DataStreams/BlockIO.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Interpreters/Quota.h>
#include <DB/Interpreters/executeQuery.h>

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