mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
Merge branch 'master' of https://github.com/yandex/ClickHouse into CLICKHOUSE-2720
This commit is contained in:
commit
ddc14c8c02
@ -18,7 +18,7 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <boost/range/iterator_range_core.hpp>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
@ -23,6 +23,8 @@ struct CollectTables;
|
||||
* SELECT array FROM t ARRAY JOIN array array -> array
|
||||
* SELECT nested.elem FROM t ARRAY JOIN nested nested -> nested
|
||||
* SELECT elem FROM t ARRAY JOIN [1, 2, 3] AS elem elem -> [1, 2, 3]
|
||||
*
|
||||
* Does not analyze arrayJoin functions.
|
||||
*/
|
||||
struct AnalyzeArrayJoins
|
||||
{
|
||||
|
@ -36,7 +36,7 @@ try
|
||||
auto system_database = std::make_shared<DatabaseMemory>("system");
|
||||
context.addDatabase("system", system_database);
|
||||
system_database->attachTable("one", StorageSystemOne::create("one"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
|
||||
context.setCurrentDatabase("system");
|
||||
|
||||
AnalyzeLambdas analyze_lambdas;
|
||||
|
@ -33,7 +33,7 @@ try
|
||||
context.addDatabase("system", system_database);
|
||||
context.setCurrentDatabase("system");
|
||||
system_database->attachTable("one", StorageSystemOne::create("one"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
|
||||
|
||||
AnalyzeResultOfQuery analyzer;
|
||||
analyzer.process(ast, context);
|
||||
|
@ -34,7 +34,7 @@ try
|
||||
context.addDatabase("system", system_database);
|
||||
context.setCurrentDatabase("system");
|
||||
system_database->attachTable("one", StorageSystemOne::create("one"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
|
||||
|
||||
CollectAliases collect_aliases;
|
||||
collect_aliases.process(ast);
|
||||
|
@ -39,7 +39,7 @@ try
|
||||
auto system_database = std::make_shared<DatabaseMemory>("system");
|
||||
context.addDatabase("system", system_database);
|
||||
system_database->attachTable("one", StorageSystemOne::create("one"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
|
||||
context.setCurrentDatabase("system");
|
||||
|
||||
AnalyzeLambdas analyze_lambdas;
|
||||
|
@ -40,7 +40,7 @@ try
|
||||
auto system_database = std::make_shared<DatabaseMemory>("system");
|
||||
context.addDatabase("system", system_database);
|
||||
system_database->attachTable("one", StorageSystemOne::create("one"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
|
||||
context.setCurrentDatabase("system");
|
||||
|
||||
AnalyzeLambdas analyze_lambdas;
|
||||
|
@ -10,7 +10,7 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <ext/enumerate.hpp>
|
||||
#include <ext/enumerate.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <ext/map.hpp>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/map.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -11,7 +11,7 @@
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
|
||||
#include <ext/bit_cast.hpp>
|
||||
#include <ext/bit_cast.h>
|
||||
|
||||
#if __SSE2__
|
||||
#include <emmintrin.h>
|
||||
|
@ -10,7 +10,7 @@
|
||||
#include <boost/intrusive/list.hpp>
|
||||
#include <boost/intrusive/set.hpp>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <ext/scope_guard.hpp>
|
||||
#include <ext/scope_guard.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
@ -198,6 +198,7 @@ public:
|
||||
/// Create table
|
||||
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>(sample_block.getColumnsList());
|
||||
StoragePtr storage = StorageMemory::create(data.second, columns);
|
||||
storage->startup();
|
||||
context.addExternalTable(data.second, storage);
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr(), context.getSettingsRef());
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <climits>
|
||||
#include <random>
|
||||
#include <common/Types.h>
|
||||
#include <ext/scope_guard.hpp>
|
||||
#include <ext/scope_guard.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Common/PoolBase.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <cstdint>
|
||||
#include <type_traits>
|
||||
|
||||
#include <ext/bit_cast.hpp>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Defines.h>
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <map>
|
||||
#include <tuple>
|
||||
#include <mutex>
|
||||
#include <ext/function_traits.hpp>
|
||||
#include <ext/function_traits.h>
|
||||
|
||||
|
||||
/** The simplest cache for a free function.
|
||||
|
@ -2,9 +2,9 @@
|
||||
|
||||
#include <Common/Arena.h>
|
||||
#include <common/likely.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/size.hpp>
|
||||
#include <ext/bit_cast.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <ext/size.h>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <cstdlib>
|
||||
#include <memory>
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
#include <Poco/Unicode.h>
|
||||
#include <stdint.h>
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
#include <Poco/Unicode.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <stdint.h>
|
||||
#include <string.h>
|
||||
|
||||
|
@ -11,8 +11,8 @@
|
||||
#include <memory>
|
||||
#include <array>
|
||||
#include <sys/resource.h>
|
||||
#include <ext/bit_cast.hpp>
|
||||
#include <ext/size.hpp>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <ext/size.h>
|
||||
#include <Common/Arena.h>
|
||||
|
||||
#include <Core/StringRef.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#if !defined(__APPLE__) && !defined(__FreeBSD__)
|
||||
#include <malloc.h>
|
||||
#endif
|
||||
#include <ext/bit_cast.hpp>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <Common/RadixSort.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <DataStreams/MaterializingBlockOutputStream.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -42,7 +42,7 @@ try
|
||||
chain.finalize();
|
||||
ExpressionActionsPtr expression = chain.getLastActions();
|
||||
|
||||
StoragePtr table = StorageSystemNumbers::create("Numbers");
|
||||
StoragePtr table = StorageSystemNumbers::create("numbers", false);
|
||||
|
||||
Names column_names;
|
||||
column_names.push_back("number");
|
||||
|
@ -48,7 +48,7 @@ try
|
||||
chain.finalize();
|
||||
ExpressionActionsPtr expression = chain.getLastActions();
|
||||
|
||||
StoragePtr table = StorageSystemNumbers::create("Numbers");
|
||||
StoragePtr table = StorageSystemNumbers::create("numbers", false);
|
||||
|
||||
Names column_names;
|
||||
column_names.push_back("number");
|
||||
|
@ -105,7 +105,9 @@ int main(int argc, char ** argv)
|
||||
|
||||
/// create an object of an existing hit log table
|
||||
|
||||
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list));
|
||||
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list),
|
||||
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
|
||||
table->startup();
|
||||
|
||||
/// read from it, apply the expression, filter, and write in tsv form to the console
|
||||
|
||||
|
@ -65,7 +65,7 @@ try
|
||||
chain.finalize();
|
||||
ExpressionActionsPtr expression = chain.getLastActions();
|
||||
|
||||
StoragePtr table = StorageSystemNumbers::create("Numbers");
|
||||
StoragePtr table = StorageSystemNumbers::create("numbers", false);
|
||||
|
||||
Names column_names;
|
||||
column_names.push_back("number");
|
||||
|
@ -95,7 +95,9 @@ try
|
||||
|
||||
/// create an object of an existing hit log table
|
||||
|
||||
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list));
|
||||
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list),
|
||||
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
|
||||
table->startup();
|
||||
|
||||
/// read from it
|
||||
if (argc == 2 && 0 == strcmp(argv[1], "read"))
|
||||
|
@ -107,7 +107,9 @@ try
|
||||
|
||||
/// create an object of an existing hit log table
|
||||
|
||||
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list));
|
||||
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list),
|
||||
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
|
||||
table->startup();
|
||||
|
||||
/// read from it, sort it, and write it in tsv form to the console
|
||||
|
||||
|
@ -25,7 +25,7 @@ using namespace DB;
|
||||
void test1()
|
||||
{
|
||||
Context context;
|
||||
StoragePtr table = StorageSystemNumbers::create("numbers");
|
||||
StoragePtr table = StorageSystemNumbers::create("numbers", false);
|
||||
|
||||
Names column_names;
|
||||
column_names.push_back("number");
|
||||
@ -55,7 +55,7 @@ void test1()
|
||||
void test2()
|
||||
{
|
||||
Context context;
|
||||
StoragePtr table = StorageSystemNumbers::create("numbers");
|
||||
StoragePtr table = StorageSystemNumbers::create("numbers", false);
|
||||
|
||||
Names column_names;
|
||||
column_names.push_back("number");
|
||||
|
@ -125,16 +125,16 @@ void DataTypeArray::serializeBinaryBulk(const IColumn & column, WriteBuffer & os
|
||||
}
|
||||
|
||||
|
||||
void DataTypeArray::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const
|
||||
void DataTypeArray::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double) const
|
||||
{
|
||||
ColumnArray & column_array = typeid_cast<ColumnArray &>(column);
|
||||
ColumnArray::Offsets_t & offsets = column_array.getOffsets();
|
||||
IColumn & nested_column = column_array.getData();
|
||||
|
||||
/// Number of values correlated with `offsets` must be read.
|
||||
/// Number of values corresponding with `offsets` must be read.
|
||||
size_t last_offset = (offsets.empty() ? 0 : offsets.back());
|
||||
if (last_offset < nested_column.size())
|
||||
throw Exception("Nested column longer than last offset", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR);
|
||||
size_t nested_limit = last_offset - nested_column.size();
|
||||
nested->deserializeBinaryBulk(nested_column, istr, nested_limit, 0);
|
||||
|
||||
|
@ -26,7 +26,7 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
|
||||
#include <ext/map.hpp>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -184,11 +184,11 @@ void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr,
|
||||
}
|
||||
else
|
||||
{
|
||||
/** A small heuristic to evaluate that there are a lot of empty lines in the column.
|
||||
/** A small heuristic to evaluate that there are a lot of empty strings in the column.
|
||||
* In this case, to save RAM, we will say that the average size of the value is small.
|
||||
*/
|
||||
if (istr.position() + sizeof(UInt32) <= istr.buffer().end()
|
||||
&& unalignedLoad<UInt32>(istr.position()) == 0) /// The first 4 rows are in the buffer and are empty.
|
||||
&& unalignedLoad<UInt32>(istr.position()) == 0) /// The first 4 strings are in the buffer and are empty.
|
||||
{
|
||||
avg_chars_size = 1;
|
||||
}
|
||||
|
@ -4,9 +4,9 @@
|
||||
#include <DataStreams/NativeBlockOutputStream.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
||||
#include <ext/map.hpp>
|
||||
#include <ext/enumerate.hpp>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/map.h>
|
||||
#include <ext/enumerate.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNull.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <ext/size.hpp>
|
||||
#include <ext/size.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -414,6 +414,8 @@ StoragePtr DatabaseCloud::tryGetTable(const String & table_name)
|
||||
definition, name, data_path, context, false,
|
||||
"in zookeeper node " + zookeeper_path + "/table_definitions/" + hashToHex(table_hash));
|
||||
|
||||
table->startup();
|
||||
|
||||
local_tables_cache.emplace(table_name, table);
|
||||
return table;
|
||||
}
|
||||
|
@ -181,6 +181,60 @@ void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, b
|
||||
task();
|
||||
}
|
||||
|
||||
if (thread_pool)
|
||||
thread_pool->wait();
|
||||
|
||||
/// After all tables was basically initialized, startup them.
|
||||
startupTables(thread_pool);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOrdinary::startupTables(ThreadPool * thread_pool)
|
||||
{
|
||||
LOG_INFO(log, "Starting up tables.");
|
||||
|
||||
StopwatchWithLock watch;
|
||||
std::atomic<size_t> tables_processed {0};
|
||||
size_t total_tables = tables.size();
|
||||
|
||||
auto task_function = [&](Tables::iterator begin, Tables::iterator end)
|
||||
{
|
||||
for (Tables::iterator it = begin; it != end; ++it)
|
||||
{
|
||||
if ((++tables_processed) % PRINT_MESSAGE_EACH_N_TABLES == 0
|
||||
|| watch.lockTestAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
|
||||
watch.restart();
|
||||
}
|
||||
|
||||
it->second->startup();
|
||||
}
|
||||
};
|
||||
|
||||
const size_t bunch_size = TABLES_PARALLEL_LOAD_BUNCH_SIZE;
|
||||
size_t num_bunches = (total_tables + bunch_size - 1) / bunch_size;
|
||||
|
||||
Tables::iterator begin = tables.begin();
|
||||
for (size_t i = 0; i < num_bunches; ++i)
|
||||
{
|
||||
auto end = begin;
|
||||
|
||||
if (i + 1 == num_bunches)
|
||||
end = tables.end();
|
||||
else
|
||||
std::advance(end, bunch_size);
|
||||
|
||||
auto task = std::bind(task_function, begin, end);
|
||||
|
||||
if (thread_pool)
|
||||
thread_pool->schedule(task);
|
||||
else
|
||||
task();
|
||||
|
||||
begin = end;
|
||||
}
|
||||
|
||||
if (thread_pool)
|
||||
thread_pool->wait();
|
||||
}
|
||||
|
@ -45,6 +45,9 @@ public:
|
||||
const NamesAndTypesList & alias_columns,
|
||||
const ColumnDefaults & column_defaults,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
private:
|
||||
void startupTables(ThreadPool * thread_pool);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -24,6 +24,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query);
|
||||
/** Create a table by its definition, without using InterpreterCreateQuery.
|
||||
* (InterpreterCreateQuery has more complex functionality, and it can not be used if the database has not been created yet)
|
||||
* Returns the table name and the table itself.
|
||||
* You must subsequently call IStorage::startup method to use the table.
|
||||
*/
|
||||
std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
const String & definition,
|
||||
|
@ -8,9 +8,9 @@
|
||||
#include <Common/ProfilingScopedRWLock.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <ext/size.hpp>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/map.hpp>
|
||||
#include <ext/size.h>
|
||||
#include <ext/range.h>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Common/ArenaWithFreeLists.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <ext/bit_cast.hpp>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <Poco/RWLock.h>
|
||||
#include <cmath>
|
||||
#include <atomic>
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <memory>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Common/ProfilingScopedRWLock.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -9,9 +9,9 @@
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Core/StringRef.h>
|
||||
#include <ext/scope_guard.hpp>
|
||||
#include <ext/bit_cast.hpp>
|
||||
#include <ext/map.hpp>
|
||||
#include <ext/scope_guard.h>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <ext/map.h>
|
||||
#include <Poco/RWLock.h>
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <ext/map.hpp>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/map.h>
|
||||
#include <ext/range.h>
|
||||
#include <Dictionaries/ComplexKeyHashedDictionary.h>
|
||||
|
||||
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <tuple>
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <numeric>
|
||||
#include <vector>
|
||||
#include <string>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <boost/range/join.hpp>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <Dictionaries/ExternalResultDescription.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
@ -5,8 +5,8 @@
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/size.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <ext/size.h>
|
||||
#include <atomic>
|
||||
#include <vector>
|
||||
#include <tuple>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <ext/size.hpp>
|
||||
#include <ext/size.h>
|
||||
#include <Dictionaries/HashedDictionary.h>
|
||||
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <tuple>
|
||||
|
@ -17,7 +17,7 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <Core/FieldVisitors.h>
|
||||
|
||||
|
||||
|
@ -15,7 +15,7 @@
|
||||
#include <Dictionaries/MongoDBDictionarySource.h>
|
||||
#include <Dictionaries/MongoDBBlockInputStream.h>
|
||||
#include <Core/FieldVisitors.h>
|
||||
#include <ext/enumerate.hpp>
|
||||
#include <ext/enumerate.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Dictionaries/MySQLBlockInputStream.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <vector>
|
||||
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <Dictionaries/IDictionarySource.h>
|
||||
#include <Dictionaries/ExternalQueryBuilder.h>
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <mysqlxx/PoolWithFailover.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <vector>
|
||||
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <tuple>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <ext/map.hpp>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/map.h>
|
||||
#include <ext/range.h>
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
#include <Poco/ByteOrder.h>
|
||||
#include <Dictionaries/TrieDictionary.h>
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <btrie.h>
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
|
@ -18,7 +18,7 @@
|
||||
#include <Functions/ObjectPool.h>
|
||||
#include <Common/StringUtils.h>
|
||||
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
|
||||
#include <unordered_map>
|
||||
#include <numeric>
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
|
||||
#include <unicode/ucnv.h>
|
||||
|
||||
|
@ -17,7 +17,7 @@
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#include <arpa/inet.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <array>
|
||||
|
||||
|
||||
|
@ -1,8 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/enumerate.hpp>
|
||||
#include <ext/collection_cast.hpp>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/enumerate.h>
|
||||
#include <ext/collection_cast.h>
|
||||
#include <ext/range.h>
|
||||
#include <type_traits>
|
||||
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
|
@ -25,7 +25,7 @@
|
||||
#include <Dictionaries/RangeHashedDictionary.h>
|
||||
#include <Dictionaries/TrieDictionary.h>
|
||||
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <math.h>
|
||||
#include <array>
|
||||
|
||||
|
@ -25,7 +25,7 @@
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <Functions/FunctionsMiscellaneous.h>
|
||||
|
||||
#include <cmath>
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <Poco/Net/DNS.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Functions/FunctionsString.h>
|
||||
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
@ -11,10 +11,10 @@ namespace DB
|
||||
struct AggregateDescription
|
||||
{
|
||||
AggregateFunctionPtr function;
|
||||
Array parameters; /// Параметры (параметрической) агрегатной функции.
|
||||
Array parameters; /// Parameters of the (parametric) aggregate function.
|
||||
ColumnNumbers arguments;
|
||||
Names argument_names; /// Используются, если arguments не заданы.
|
||||
String column_name; /// Какое имя использовать для столбца со значениями агрегатной функции
|
||||
Names argument_names; /// used if no `arguments` are specified.
|
||||
String column_name; /// What name to use for a column with aggregate function values
|
||||
};
|
||||
|
||||
using AggregateDescriptions = std::vector<AggregateDescription>;
|
||||
|
@ -181,7 +181,7 @@ static inline UInt128 ALWAYS_INLINE hash128(
|
||||
|
||||
for (size_t j = 0; j < keys_size; ++j)
|
||||
{
|
||||
/// Хэшируем ключ.
|
||||
/// Hashes the key.
|
||||
keys[j] = key_columns[j]->getDataAtWithTerminatingZero(i);
|
||||
hash.update(keys[j].data, keys[j].size);
|
||||
}
|
||||
@ -208,18 +208,18 @@ static inline UInt128 ALWAYS_INLINE hash128(
|
||||
}
|
||||
|
||||
|
||||
/// Скопировать ключи в пул. Потом разместить в пуле StringRef-ы на них и вернуть указатель на первый.
|
||||
/// Copy keys to the pool. Then put into pool StringRefs to them and return the pointer to the first.
|
||||
static inline StringRef * ALWAYS_INLINE placeKeysInPool(
|
||||
size_t i, size_t keys_size, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
for (size_t j = 0; j < keys_size; ++j)
|
||||
{
|
||||
char * place = pool.alloc(keys[j].size);
|
||||
memcpy(place, keys[j].data, keys[j].size); /// TODO padding в Arena и memcpySmall
|
||||
memcpy(place, keys[j].data, keys[j].size); /// TODO padding in Arena and memcpySmall
|
||||
keys[j].data = place;
|
||||
}
|
||||
|
||||
/// Размещаем в пуле StringRef-ы на только что скопированные ключи.
|
||||
/// Place the StringRefs on the newly copied keys in the pool.
|
||||
char * res = pool.alloc(keys_size * sizeof(StringRef));
|
||||
memcpy(res, &keys[0], keys_size * sizeof(StringRef));
|
||||
|
||||
@ -227,7 +227,7 @@ static inline StringRef * ALWAYS_INLINE placeKeysInPool(
|
||||
}
|
||||
|
||||
|
||||
/// Скопировать ключи в пул. Потом разместить в пуле StringRef-ы на них и вернуть указатель на первый.
|
||||
/// Copy keys to the pool. Then put into pool StringRefs to them and return the pointer to the first.
|
||||
static inline StringRef * ALWAYS_INLINE extractKeysAndPlaceInPool(
|
||||
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
@ -239,7 +239,7 @@ static inline StringRef * ALWAYS_INLINE extractKeysAndPlaceInPool(
|
||||
keys[j].data = place;
|
||||
}
|
||||
|
||||
/// Размещаем в пуле StringRef-ы на только что скопированные ключи.
|
||||
/// Place the StringRefs on the newly copied keys in the pool.
|
||||
char * res = pool.alloc(keys_size * sizeof(StringRef));
|
||||
memcpy(res, &keys[0], keys_size * sizeof(StringRef));
|
||||
|
||||
@ -280,14 +280,14 @@ inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous(
|
||||
place += keys[j].size;
|
||||
}
|
||||
|
||||
/// Размещаем в пуле StringRef-ы на только что скопированные ключи.
|
||||
/// Place the StringRefs on the newly copied keys in the pool.
|
||||
memcpy(place, &keys[0], keys_size * sizeof(StringRef));
|
||||
|
||||
return {res, sum_keys_size};
|
||||
}
|
||||
|
||||
|
||||
/** Сериализовать ключи в непрерывный кусок памяти.
|
||||
/** Serialize keys into a continuous chunk of memory.
|
||||
*/
|
||||
static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous(
|
||||
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
|
||||
|
@ -39,23 +39,23 @@ namespace ErrorCodes
|
||||
class IBlockOutputStream;
|
||||
|
||||
|
||||
/** Разные структуры данных, которые могут использоваться для агрегации
|
||||
* Для эффективности, сами данные для агрегации кладутся в пул.
|
||||
* Владение данными (состояний агрегатных функций) и пулом
|
||||
* захватывается позднее - в функции convertToBlocks, объектом ColumnAggregateFunction.
|
||||
/** Different data structures that can be used for aggregation
|
||||
* For efficiency, the aggregation data itself is put into the pool.
|
||||
* Data and pool ownership (states of aggregate functions)
|
||||
* is acquired later - in `convertToBlocks` function, by the ColumnAggregateFunction object.
|
||||
*
|
||||
* Большинство структур данных существует в двух вариантах: обычном и двухуровневом (TwoLevel).
|
||||
* Двухуровневая хэш-таблица работает чуть медленнее при маленьком количестве различных ключей,
|
||||
* но при большом количестве различных ключей лучше масштабируется, так как позволяет
|
||||
* распараллелить некоторые операции (слияние, пост-обработку) естественным образом.
|
||||
* Most data structures exist in two versions: normal and two-level (TwoLevel).
|
||||
* A two-level hash table works a little slower with a small number of different keys,
|
||||
* but with a large number of different keys scales better, because it allows
|
||||
* parallelize some operations (merging, post-processing) in a natural way.
|
||||
*
|
||||
* Чтобы обеспечить эффективную работу в большом диапазоне условий,
|
||||
* сначала используются одноуровневые хэш-таблицы,
|
||||
* а при достижении количеством различных ключей достаточно большого размера,
|
||||
* они конвертируются в двухуровневые.
|
||||
* To ensure efficient work over a wide range of conditions,
|
||||
* first single-level hash tables are used,
|
||||
* and when the number of different keys is large enough,
|
||||
* they are converted to two-level ones.
|
||||
*
|
||||
* PS. Существует много различных подходов к эффективной реализации параллельной и распределённой агрегации,
|
||||
* лучшим образом подходящих для разных случаев, и этот подход - всего лишь один из них, выбранный по совокупности причин.
|
||||
* PS. There are many different approaches to the effective implementation of parallel and distributed aggregation,
|
||||
* best suited for different cases, and this approach is just one of them, chosen for a combination of reasons.
|
||||
*/
|
||||
|
||||
using AggregatedDataWithoutKey = AggregateDataPtr;
|
||||
@ -88,8 +88,8 @@ using AggregatedDataWithKeys128Hash64 = HashMap<UInt128, AggregateDataPtr, UInt1
|
||||
using AggregatedDataWithKeys256Hash64 = HashMap<UInt256, AggregateDataPtr, UInt256Hash>;
|
||||
|
||||
|
||||
/// Для случая, когда есть один числовой ключ.
|
||||
template <typename FieldType, typename TData> /// UInt8/16/32/64 для любых типов соответствующей битности.
|
||||
/// For the case where there is one numeric key.
|
||||
template <typename FieldType, typename TData> /// UInt8/16/32/64 for any type with corresponding bit width.
|
||||
struct AggregationMethodOneNumber
|
||||
{
|
||||
using Data = TData;
|
||||
@ -105,51 +105,51 @@ struct AggregationMethodOneNumber
|
||||
template <typename Other>
|
||||
AggregationMethodOneNumber(const Other & other) : data(other.data) {}
|
||||
|
||||
/// Для использования одного Method в разных потоках, используйте разные State.
|
||||
/// To use one `Method` in different threads, use different `State`.
|
||||
struct State
|
||||
{
|
||||
const FieldType * vec;
|
||||
|
||||
/** Вызывается в начале обработки каждого блока.
|
||||
* Устанавливает переменные, необходимые для остальных методов, вызываемых во внутренних циклах.
|
||||
/** Called at the start of each block processing.
|
||||
* Sets the variables needed for the other methods called in internal loops.
|
||||
*/
|
||||
void init(ConstColumnPlainPtrs & key_columns)
|
||||
{
|
||||
vec = &static_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData()[0];
|
||||
}
|
||||
|
||||
/// Достать из ключевых столбцов ключ для вставки в хэш-таблицу.
|
||||
/// Get the key from the key columns for insertion into the hash table.
|
||||
Key getKey(
|
||||
const ConstColumnPlainPtrs & key_columns, /// Ключевые столбцы.
|
||||
size_t keys_size, /// Количество ключевых столбцов.
|
||||
size_t i, /// Из какой строки блока достать ключ.
|
||||
const Sizes & key_sizes, /// Если ключи фиксированной длины - их длины. Не используется в методах агрегации по ключам переменной длины.
|
||||
StringRefs & keys, /// Сюда могут быть записаны ссылки на данные ключей в столбцах. Они могут быть использованы в дальнейшем.
|
||||
const ConstColumnPlainPtrs & key_columns, /// Key columns.
|
||||
size_t keys_size, /// Number of key columns.
|
||||
size_t i, /// From which row of the block, get the key.
|
||||
const Sizes & key_sizes, /// If the keys of a fixed length - their lengths. It is not used in aggregation methods for variable length keys.
|
||||
StringRefs & keys, /// Here references to key data in columns can be written. They can be used in the future.
|
||||
Arena & pool) const
|
||||
{
|
||||
return unionCastToUInt64(vec[i]);
|
||||
}
|
||||
};
|
||||
|
||||
/// Из значения в хэш-таблице получить AggregateDataPtr.
|
||||
/// From the value in the hash table, get AggregateDataPtr.
|
||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||
|
||||
/** Разместить дополнительные данные, если это необходимо, в случае, когда в хэш-таблицу был вставлен новый ключ.
|
||||
/** Place additional data, if necessary, in case a new key was inserted into the hash table.
|
||||
*/
|
||||
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
}
|
||||
|
||||
/** Действие, которое нужно сделать, если ключ не новый. Например, откатить выделение памяти в пуле.
|
||||
/** The action to be taken if the key is not new. For example, roll back the memory allocation in the pool.
|
||||
*/
|
||||
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
|
||||
|
||||
/** Не использовать оптимизацию для идущих подряд ключей.
|
||||
/** Do not use optimization for consecutive keys.
|
||||
*/
|
||||
static const bool no_consecutive_keys_optimization = false;
|
||||
|
||||
/** Вставить ключ из хэш-таблицы в столбцы.
|
||||
/** Insert the key from the hash table into columns.
|
||||
*/
|
||||
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
|
||||
{
|
||||
@ -158,7 +158,7 @@ struct AggregationMethodOneNumber
|
||||
};
|
||||
|
||||
|
||||
/// Для случая, когда есть один строковый ключ.
|
||||
/// For the case where there is one string key.
|
||||
template <typename TData>
|
||||
struct AggregationMethodString
|
||||
{
|
||||
@ -221,7 +221,7 @@ struct AggregationMethodString
|
||||
};
|
||||
|
||||
|
||||
/// Для случая, когда есть один строковый ключ фиксированной длины.
|
||||
/// For the case where there is one fixed-length string key.
|
||||
template <typename TData>
|
||||
struct AggregationMethodFixedString
|
||||
{
|
||||
@ -378,7 +378,7 @@ protected:
|
||||
|
||||
}
|
||||
|
||||
/// Для случая, когда все ключи фиксированной длины, и они помещаются в N (например, 128) бит.
|
||||
/// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits.
|
||||
template <typename TData, bool has_nullable_keys_ = false>
|
||||
struct AggregationMethodKeysFixed
|
||||
{
|
||||
@ -487,7 +487,7 @@ struct AggregationMethodKeysFixed
|
||||
};
|
||||
|
||||
|
||||
/// Агрегирует по конкатенации ключей. (При этом, строки, содержащие нули посередине, могут склеиться.)
|
||||
/// Aggregates by key concatenation. (In this case, strings containing zeros in the middle can stick together.)
|
||||
template <typename TData>
|
||||
struct AggregationMethodConcat
|
||||
{
|
||||
@ -534,7 +534,7 @@ struct AggregationMethodConcat
|
||||
pool.rollback(key.size + keys.size() * sizeof(keys[0]));
|
||||
}
|
||||
|
||||
/// Если ключ уже был, то он удаляется из пула (затирается), и сравнить с ним следующий ключ уже нельзя.
|
||||
/// If the key already was, then it is removed from the pool (overwritten), and the next key can not be compared with it.
|
||||
static const bool no_consecutive_keys_optimization = true;
|
||||
|
||||
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
|
||||
@ -546,14 +546,14 @@ private:
|
||||
/// Insert the values of the specified keys into the corresponding columns.
|
||||
static void insertKeyIntoColumnsImpl(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
|
||||
{
|
||||
/// См. функцию extractKeysAndPlaceInPoolContiguous.
|
||||
/// See function extractKeysAndPlaceInPoolContiguous.
|
||||
const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size);
|
||||
|
||||
if (unlikely(0 == value.first.size))
|
||||
{
|
||||
/** Исправление, если все ключи - пустые массивы. Для них в хэш-таблицу записывается StringRef нулевой длины, но с ненулевым указателем.
|
||||
* Но при вставке в хэш-таблицу, такой StringRef оказывается равен другому ключу нулевой длины,
|
||||
* у которого указатель на данные может быть любым мусором и использовать его нельзя.
|
||||
/** Fix if all keys are empty arrays. For them, a zero-length StringRef is written to the hash table, but with a non-zero pointer.
|
||||
* But when inserted into a hash table, this StringRef occurs equal to another key of zero length,
|
||||
* whose data pointer can be any garbage and can not be used.
|
||||
*/
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
key_columns[i]->insertDefault();
|
||||
@ -567,11 +567,11 @@ private:
|
||||
};
|
||||
|
||||
|
||||
/** Агрегирует по конкатенации сериализованных значений ключей.
|
||||
* Похож на AggregationMethodConcat, но подходит, например, для массивов строк или нескольких массивов.
|
||||
* Сериализованное значение отличается тем, что позволяет однозначно его десериализовать, имея только позицию, с которой оно начинается.
|
||||
* То есть, например, для строк, оно содержит сначала сериализованную длину строки, а потом байты.
|
||||
* Поэтому, при агрегации по нескольким строкам, неоднозначностей не возникает.
|
||||
/** Aggregates by concatenating serialized key values.
|
||||
* Similar to AggregationMethodConcat, but it is suitable, for example, for arrays of strings or multiple arrays.
|
||||
* The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts.
|
||||
* That is, for example, for strings, it contains first the serialized length of the string, and then the bytes.
|
||||
* Therefore, when aggregating by several strings, there is no ambiguity.
|
||||
*/
|
||||
template <typename TData>
|
||||
struct AggregationMethodSerialized
|
||||
@ -619,7 +619,7 @@ struct AggregationMethodSerialized
|
||||
pool.rollback(key.size);
|
||||
}
|
||||
|
||||
/// Если ключ уже был, то он удаляется из пула (затирается), и сравнить с ним следующий ключ уже нельзя.
|
||||
/// If the key already was, it is removed from the pool (overwritten), and the next key can not be compared with it.
|
||||
static const bool no_consecutive_keys_optimization = true;
|
||||
|
||||
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
|
||||
@ -631,7 +631,7 @@ struct AggregationMethodSerialized
|
||||
};
|
||||
|
||||
|
||||
/// Для остальных случаев. Агрегирует по 128 битному хэшу от ключа.
|
||||
/// For other cases. Aggregates by 128-bit hash from the key.
|
||||
template <typename TData>
|
||||
struct AggregationMethodHashed
|
||||
{
|
||||
@ -690,33 +690,33 @@ class Aggregator;
|
||||
|
||||
struct AggregatedDataVariants : private boost::noncopyable
|
||||
{
|
||||
/** Работа с состояниями агрегатных функций в пуле устроена следующим (неудобным) образом:
|
||||
* - при агрегации, состояния создаются в пуле с помощью функции IAggregateFunction::create (внутри - placement new произвольной структуры);
|
||||
* - они должны быть затем уничтожены с помощью IAggregateFunction::destroy (внутри - вызов деструктора произвольной структуры);
|
||||
* - если агрегация завершена, то, в функции Aggregator::convertToBlocks, указатели на состояния агрегатных функций
|
||||
* записываются в ColumnAggregateFunction; ColumnAggregateFunction "захватывает владение" ими, то есть - вызывает destroy в своём деструкторе.
|
||||
* - если при агрегации, до вызова Aggregator::convertToBlocks вылетело исключение,
|
||||
* то состояния агрегатных функций всё-равно должны быть уничтожены,
|
||||
* иначе для сложных состояний (наприемер, AggregateFunctionUniq), будут утечки памяти;
|
||||
* - чтобы, в этом случае, уничтожить состояния, в деструкторе вызывается метод Aggregator::destroyAggregateStates,
|
||||
* но только если переменная aggregator (см. ниже) не nullptr;
|
||||
* - то есть, пока вы не передали владение состояниями агрегатных функций в ColumnAggregateFunction, установите переменную aggregator,
|
||||
* чтобы при возникновении исключения, состояния были корректно уничтожены.
|
||||
/** Working with states of aggregate functions in the pool is arranged in the following (inconvenient) way:
|
||||
* - when aggregating, states are created in the pool using IAggregateFunction::create (inside - `placement new` of arbitrary structure);
|
||||
* - they must then be destroyed using IAggregateFunction::destroy (inside - calling the destructor of arbitrary structure);
|
||||
* - if aggregation is complete, then, in the Aggregator::convertToBlocks function, pointers to the states of aggregate functions
|
||||
* are written to ColumnAggregateFunction; ColumnAggregateFunction "acquires ownership" of them, that is - calls `destroy` in its destructor.
|
||||
* - if during the aggregation, before call to Aggregator::convertToBlocks, an exception was thrown,
|
||||
* then the states of aggregate functions must still be destroyed,
|
||||
* otherwise, for complex states (eg, AggregateFunctionUniq), there will be memory leaks;
|
||||
* - in this case, to destroy states, the destructor calls Aggregator::destroyAggregateStates method,
|
||||
* but only if the variable aggregator (see below) is not nullptr;
|
||||
* - that is, until you transfer ownership of the aggregate function states in the ColumnAggregateFunction, set the variable `aggregator`,
|
||||
* so that when an exception occurs, the states are correctly destroyed.
|
||||
*
|
||||
* PS. Это можно исправить, сделав пул, который знает о том, какие состояния агрегатных функций и в каком порядке в него уложены, и умеет сам их уничтожать.
|
||||
* Но это вряд ли можно просто сделать, так как в этот же пул планируется класть строки переменной длины.
|
||||
* В этом случае, пул не сможет знать, по каким смещениям хранятся объекты.
|
||||
* PS. This can be corrected by making a pool that knows about which states of aggregate functions and in which order are put in it, and knows how to destroy them.
|
||||
* But this can hardly be done simply because it is planned to put variable-length strings into the same pool.
|
||||
* In this case, the pool will not be able to know with what offsets objects are stored.
|
||||
*/
|
||||
Aggregator * aggregator = nullptr;
|
||||
|
||||
size_t keys_size; /// Количество ключей NOTE нужно ли это поле?
|
||||
Sizes key_sizes; /// Размеры ключей, если ключи фиксированной длины
|
||||
size_t keys_size; /// Number of keys. NOTE do we need this field?
|
||||
Sizes key_sizes; /// Dimensions of keys, if keys of fixed length
|
||||
|
||||
/// Пулы для состояний агрегатных функций. Владение потом будет передано в ColumnAggregateFunction.
|
||||
/// Pools for states of aggregate functions. Ownership will be later transferred to ColumnAggregateFunction.
|
||||
Arenas aggregates_pools;
|
||||
Arena * aggregates_pool; /// Пул, который сейчас используется для аллокации.
|
||||
Arena * aggregates_pool; /// The pool that is currently used for allocation.
|
||||
|
||||
/** Специализация для случая, когда ключи отсутствуют, и для ключей, не попавших в max_rows_to_group_by.
|
||||
/** Specialization for the case when there are no keys, and for keys not fitted into max_rows_to_group_by.
|
||||
*/
|
||||
AggregatedDataWithoutKey without_key = nullptr;
|
||||
|
||||
@ -757,7 +757,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128TwoLevel, true>> nullable_keys128_two_level;
|
||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel, true>> nullable_keys256_two_level;
|
||||
|
||||
/// В этом и подобных макросах, вариант without_key не учитывается.
|
||||
/// In this and similar macros, the option without_key is not considered.
|
||||
#define APPLY_FOR_AGGREGATED_VARIANTS(M) \
|
||||
M(key8, false) \
|
||||
M(key16, false) \
|
||||
@ -827,7 +827,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
type = type_;
|
||||
}
|
||||
|
||||
/// Количество строк (разных ключей).
|
||||
/// Number of rows (different keys).
|
||||
size_t size() const
|
||||
{
|
||||
switch (type)
|
||||
@ -845,7 +845,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
}
|
||||
}
|
||||
|
||||
/// Размер без учёта строчки, в которую записываются данные для расчёта TOTALS.
|
||||
/// The size without taking into account the row in which data is written for the calculation of TOTALS.
|
||||
size_t sizeWithoutOverflowRow() const
|
||||
{
|
||||
switch (type)
|
||||
@ -959,52 +959,52 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
using AggregatedDataVariantsPtr = std::shared_ptr<AggregatedDataVariants>;
|
||||
using ManyAggregatedDataVariants = std::vector<AggregatedDataVariantsPtr>;
|
||||
|
||||
/** Как считаются "тотальные" значения при наличии WITH TOTALS?
|
||||
* (Более подробно смотрите в TotalsHavingBlockInputStream.)
|
||||
/** How are "total" values calculated with WITH TOTALS?
|
||||
* (For more details, see TotalsHavingBlockInputStream.)
|
||||
*
|
||||
* В случае отсутствия group_by_overflow_mode = 'any', данные агрегируются как обычно, но состояния агрегатных функций не финализируются.
|
||||
* Позже, состояния агрегатных функций для всех строк (прошедших через HAVING) мерджатся в одну - это и будет TOTALS.
|
||||
* In the absence of group_by_overflow_mode = 'any', the data is aggregated as usual, but the states of the aggregate functions are not finalized.
|
||||
* Later, the aggregate function states for all rows (passed through HAVING) are merged into one - this will be TOTALS.
|
||||
*
|
||||
* В случае наличия group_by_overflow_mode = 'any', данные агрегируются как обычно, кроме ключей, не поместившихся в max_rows_to_group_by.
|
||||
* Для этих ключей, данные агрегируются в одну дополнительную строку - далее см. под названиями overflow_row, overflows...
|
||||
* Позже, состояния агрегатных функций для всех строк (прошедших через HAVING) мерджатся в одну,
|
||||
* а также к ним прибавляется или не прибавляется (в зависимости от настройки totals_mode) также overflow_row - это и будет TOTALS.
|
||||
* If there is group_by_overflow_mode = 'any', the data is aggregated as usual, except for the keys that did not fit in max_rows_to_group_by.
|
||||
* For these keys, the data is aggregated into one additional row - see below under the names `overflow_row`, `overflows`...
|
||||
* Later, the aggregate function states for all rows (passed through HAVING) are merged into one,
|
||||
* also overflow_row is added or not added (depending on the totals_mode setting) also - this will be TOTALS.
|
||||
*/
|
||||
|
||||
|
||||
/** Агрегирует источник блоков.
|
||||
/** Aggregates the source of the blocks.
|
||||
*/
|
||||
class Aggregator
|
||||
{
|
||||
public:
|
||||
struct Params
|
||||
{
|
||||
/// Что считать.
|
||||
/// What to count.
|
||||
Names key_names;
|
||||
ColumnNumbers keys; /// Номера столбцов - вычисляются позже.
|
||||
ColumnNumbers keys; /// The column numbers are computed later.
|
||||
AggregateDescriptions aggregates;
|
||||
size_t keys_size;
|
||||
size_t aggregates_size;
|
||||
|
||||
/// Настройки приближённого вычисления GROUP BY.
|
||||
const bool overflow_row; /// Нужно ли класть в AggregatedDataVariants::without_key агрегаты для ключей, не попавших в max_rows_to_group_by.
|
||||
/// The settings of approximate calculation of GROUP BY.
|
||||
const bool overflow_row; /// Do we need to put into AggregatedDataVariants::without_key aggregates for keys that are not in max_rows_to_group_by.
|
||||
const size_t max_rows_to_group_by;
|
||||
const OverflowMode group_by_overflow_mode;
|
||||
|
||||
/// Для динамической компиляции.
|
||||
/// For dynamic compilation.
|
||||
Compiler * compiler;
|
||||
const UInt32 min_count_to_compile;
|
||||
|
||||
/// Настройки двухуровневой агрегации (используется для большого количества ключей).
|
||||
/** При каком количестве ключей или размере состояния агрегации в байтах,
|
||||
* начинает использоваться двухуровневая агрегация. Достаточно срабатывания хотя бы одного из порогов.
|
||||
* 0 - соответствующий порог не задан.
|
||||
/// Two-level aggregation settings (used for a large number of keys).
|
||||
/** With how many keys or the size of the aggregation state in bytes,
|
||||
* two-level aggregation begins to be used. Enough to reach of at least one of the thresholds.
|
||||
* 0 - the corresponding threshold is not specified.
|
||||
*/
|
||||
const size_t group_by_two_level_threshold;
|
||||
const size_t group_by_two_level_threshold_bytes;
|
||||
|
||||
/// Настройки для сброса временных данных в файловую систему (внешняя агрегация).
|
||||
const size_t max_bytes_before_external_group_by; /// 0 - не использовать внешнюю агрегацию.
|
||||
/// Settings to flush temporary data to the file system (external aggregation).
|
||||
const size_t max_bytes_before_external_group_by; /// 0 - do not use external aggregation.
|
||||
const std::string tmp_path;
|
||||
|
||||
Params(
|
||||
@ -1024,11 +1024,11 @@ public:
|
||||
keys_size = key_names.size();
|
||||
}
|
||||
|
||||
/// Только параметры, имеющие значение при мердже.
|
||||
/// Only parameters that matter during merge.
|
||||
Params(const Names & key_names_, const AggregateDescriptions & aggregates_, bool overflow_row_)
|
||||
: Params(key_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0, 0, 0, "") {}
|
||||
|
||||
/// Вычислить номера столбцов в keys и aggregates.
|
||||
/// Compute the column numbers in `keys` and `aggregates`.
|
||||
void calculateColumnNumbers(const Block & block);
|
||||
};
|
||||
|
||||
@ -1038,38 +1038,38 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
/// Агрегировать источник. Получить результат в виде одной из структур данных.
|
||||
/// Aggregate the source. Get the result in the form of one of the data structures.
|
||||
void execute(BlockInputStreamPtr stream, AggregatedDataVariants & result);
|
||||
|
||||
using AggregateColumns = std::vector<ConstColumnPlainPtrs>;
|
||||
using AggregateColumnsData = std::vector<ColumnAggregateFunction::Container_t *>;
|
||||
using AggregateFunctionsPlainPtrs = std::vector<IAggregateFunction *>;
|
||||
|
||||
/// Обработать один блок. Вернуть false, если обработку следует прервать (при group_by_overflow_mode = 'break').
|
||||
/// Process one block. Return false if the processing should be aborted (with group_by_overflow_mode = 'break').
|
||||
bool executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
ConstColumnPlainPtrs & key_columns, AggregateColumns & aggregate_columns, /// Передаются, чтобы не создавать их заново на каждый блок
|
||||
Sizes & key_sizes, StringRefs & keys, /// - передайте соответствующие объекты, которые изначально пустые.
|
||||
ConstColumnPlainPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
|
||||
Sizes & key_sizes, StringRefs & keys, /// - pass the corresponding objects that are initially empty.
|
||||
bool & no_more_keys);
|
||||
|
||||
/** Преобразовать структуру данных агрегации в блок.
|
||||
* Если overflow_row = true, то агрегаты для строк, не попавших в max_rows_to_group_by, кладутся в первый блок.
|
||||
/** Convert the aggregation data structure into a block.
|
||||
* If overflow_row = true, then aggregates for rows that are not included in max_rows_to_group_by are put in the first block.
|
||||
*
|
||||
* Если final = false, то в качестве столбцов-агрегатов создаются ColumnAggregateFunction с состоянием вычислений,
|
||||
* которые могут быть затем объединены с другими состояниями (для распределённой обработки запроса).
|
||||
* Если final = true, то в качестве столбцов-агрегатов создаются столбцы с готовыми значениями.
|
||||
* If final = false, then ColumnAggregateFunction is created as the aggregation columns with the state of the calculations,
|
||||
* which can then be combined with other states (for distributed query processing).
|
||||
* If final = true, then columns with ready values are created as aggregate columns.
|
||||
*/
|
||||
BlocksList convertToBlocks(AggregatedDataVariants & data_variants, bool final, size_t max_threads) const;
|
||||
|
||||
/** Объединить несколько структур данных агрегации и выдать результат в виде потока блоков.
|
||||
/** Merge several aggregation data structures and output the result as a block stream.
|
||||
*/
|
||||
std::unique_ptr<IBlockInputStream> mergeAndConvertToBlocks(ManyAggregatedDataVariants & data_variants, bool final, size_t max_threads) const;
|
||||
|
||||
/** Объединить поток частично агрегированных блоков в одну структуру данных.
|
||||
* (Доагрегировать несколько блоков, которые представляют собой результат независимых агрегаций с удалённых серверов.)
|
||||
/** Merge the stream of partially aggregated blocks into one data structure.
|
||||
* (Pre-aggregate several blocks that represent the result of independent aggregations from remote servers.)
|
||||
*/
|
||||
void mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads);
|
||||
|
||||
/** Объединить несколько частично агрегированных блоков в один.
|
||||
/** Merge several partially aggregated blocks into one.
|
||||
*/
|
||||
Block mergeBlocks(BlocksList & blocks, bool final);
|
||||
|
||||
@ -1080,14 +1080,14 @@ public:
|
||||
|
||||
using CancellationHook = std::function<bool()>;
|
||||
|
||||
/** Установить функцию, которая проверяет, можно ли прервать текущую задачу.
|
||||
/** Set a function that checks whether the current task can be aborted.
|
||||
*/
|
||||
void setCancellationHook(const CancellationHook cancellation_hook);
|
||||
|
||||
/// Для IBlockInputStream.
|
||||
/// For IBlockInputStream.
|
||||
String getID() const;
|
||||
|
||||
/// Для внешней агрегации.
|
||||
/// For external aggregation.
|
||||
void writeToTemporaryFile(AggregatedDataVariants & data_variants, size_t rows);
|
||||
|
||||
bool hasTemporaryFiles() const { return !temporary_files.empty(); }
|
||||
@ -1116,14 +1116,14 @@ protected:
|
||||
|
||||
AggregateFunctionsPlainPtrs aggregate_functions;
|
||||
|
||||
/** Данный массив служит для двух целей.
|
||||
/** This array serves two purposes.
|
||||
*
|
||||
* 1. Аргументы функции собраны рядом, и их не нужно собирать из разных мест. Также массив сделан zero-terminated.
|
||||
* Внутренний цикл (для случая without_key) получается почти в два раза компактнее; прирост производительности около 30%.
|
||||
* 1. Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated.
|
||||
* The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%.
|
||||
*
|
||||
* 2. Вызов по указателю на функцию лучше, чем виртуальный вызов, потому что в случае виртуального вызова,
|
||||
* GCC 5.1.2 генерирует код, который на каждой итерации цикла заново грузит из памяти в регистр адрес функции
|
||||
* (значение по смещению в таблице виртуальных функций).
|
||||
* 2. Calling a function by pointer is better than a virtual call, because in the case of a virtual call,
|
||||
* GCC 5.1.2 generates code that, at each iteration of the loop, reloads the function address from memory into the register
|
||||
* (the offset value in the virtual function table).
|
||||
*/
|
||||
struct AggregateFunctionInstruction
|
||||
{
|
||||
@ -1135,14 +1135,14 @@ protected:
|
||||
|
||||
using AggregateFunctionInstructions = std::vector<AggregateFunctionInstruction>;
|
||||
|
||||
Sizes offsets_of_aggregate_states; /// Смещение до n-ой агрегатной функции в строке из агрегатных функций.
|
||||
size_t total_size_of_aggregate_states = 0; /// Суммарный размер строки из агрегатных функций.
|
||||
Sizes offsets_of_aggregate_states; /// The offset to the n-th aggregate function in a row of aggregate functions.
|
||||
size_t total_size_of_aggregate_states = 0; /// The total size of the row from the aggregate functions.
|
||||
bool all_aggregates_has_trivial_destructor = false;
|
||||
|
||||
/// Сколько было использовано оперативки для обработки запроса до начала обработки первого блока.
|
||||
/// How many RAM were used to process the query before processing the first block.
|
||||
Int64 memory_usage_before_aggregation = 0;
|
||||
|
||||
/// Для инициализации от первого блока при конкуррентном использовании.
|
||||
/// To initialize from the first block when used concurrently.
|
||||
bool initialized = false;
|
||||
std::mutex mutex;
|
||||
|
||||
@ -1150,56 +1150,56 @@ protected:
|
||||
|
||||
Logger * log = &Logger::get("Aggregator");
|
||||
|
||||
/** Динамически скомпилированная библиотека для агрегации, если есть.
|
||||
* Смысл динамической компиляции в том, чтобы специализировать код
|
||||
* под конкретный список агрегатных функций.
|
||||
* Это позволяет развернуть цикл по созданию и обновлению состояний агрегатных функций,
|
||||
* а также использовать вместо виртуальных вызовов inline-код.
|
||||
/** Dynamically compiled library for aggregation, if any.
|
||||
* The meaning of dynamic compilation is to specialize code
|
||||
* for a specific list of aggregate functions.
|
||||
* This allows you to expand the loop to create and update states of aggregate functions,
|
||||
* and also use inline-code instead of virtual calls.
|
||||
*/
|
||||
struct CompiledData
|
||||
{
|
||||
SharedLibraryPtr compiled_aggregator;
|
||||
|
||||
/// Получены с помощью dlsym. Нужно ещё сделать reinterpret_cast в указатель на функцию.
|
||||
/// Obtained with dlsym. It is still necessary to make reinterpret_cast to the function pointer.
|
||||
void * compiled_method_ptr = nullptr;
|
||||
void * compiled_two_level_method_ptr = nullptr;
|
||||
};
|
||||
/// shared_ptr - чтобы передавать в callback, который может пережить Aggregator.
|
||||
/// shared_ptr - to pass into a callback, that can survive Aggregator.
|
||||
std::shared_ptr<CompiledData> compiled_data { new CompiledData };
|
||||
|
||||
bool compiled_if_possible = false;
|
||||
void compileIfPossible(AggregatedDataVariants::Type type);
|
||||
|
||||
/// Возвращает true, если можно прервать текущую задачу.
|
||||
/// Returns true if you can abort the current task.
|
||||
CancellationHook isCancelled;
|
||||
|
||||
/// Для внешней агрегации.
|
||||
/// For external aggregation.
|
||||
TemporaryFiles temporary_files;
|
||||
|
||||
/** Если заданы только имена столбцов (key_names, а также aggregates[i].column_name), то вычислить номера столбцов.
|
||||
* Сформировать блок - пример результата. Он используется в методах convertToBlocks, mergeAndConvertToBlocks.
|
||||
/** If only the column names (key_names, and also aggregates[i].column_name) are specified, then calculate the column numbers.
|
||||
* Generate block - sample of the result. It is used in the convertToBlocks, mergeAndConvertToBlocks methods.
|
||||
*/
|
||||
void initialize(const Block & block);
|
||||
|
||||
/** Установить блок - пример результата,
|
||||
* только если он ещё не был установлен.
|
||||
/** Set the block - sample of the result,
|
||||
* only if it has not already been set.
|
||||
*/
|
||||
void setSampleBlock(const Block & block);
|
||||
|
||||
/** Выбрать способ агрегации на основе количества и типов ключей. */
|
||||
/** Select the aggregation method based on the number and types of keys. */
|
||||
AggregatedDataVariants::Type chooseAggregationMethod(const ConstColumnPlainPtrs & key_columns, Sizes & key_sizes) const;
|
||||
|
||||
/** Создать состояния агрегатных функций для одного ключа.
|
||||
/** Create states of aggregate functions for one key.
|
||||
*/
|
||||
void createAggregateStates(AggregateDataPtr & aggregate_data) const;
|
||||
|
||||
/** Вызвать методы destroy для состояний агрегатных функций.
|
||||
* Используется в обработчике исключений при агрегации, так как RAII в данном случае не применим.
|
||||
/** Call `destroy` methods for states of aggregate functions.
|
||||
* Used in the exception handler for aggregation, since RAII in this case is not applicable.
|
||||
*/
|
||||
void destroyAllAggregateStates(AggregatedDataVariants & result);
|
||||
|
||||
|
||||
/// Обработать один блок данных, агрегировать данные в хэш-таблицу.
|
||||
/// Process one data block, aggregate the data into a hash table.
|
||||
template <typename Method>
|
||||
void executeImpl(
|
||||
Method & method,
|
||||
@ -1212,7 +1212,7 @@ protected:
|
||||
bool no_more_keys,
|
||||
AggregateDataPtr overflow_row) const;
|
||||
|
||||
/// Специализация для конкретного значения no_more_keys.
|
||||
/// Specialization for a particular value no_more_keys.
|
||||
template <bool no_more_keys, typename Method>
|
||||
void executeImplCase(
|
||||
Method & method,
|
||||
@ -1225,7 +1225,7 @@ protected:
|
||||
StringRefs & keys,
|
||||
AggregateDataPtr overflow_row) const;
|
||||
|
||||
/// Для случая, когда нет ключей (всё агрегировать в одну строку).
|
||||
/// For case when there are no keys (all aggregate into one row).
|
||||
void executeWithoutKeyImpl(
|
||||
AggregatedDataWithoutKey & res,
|
||||
size_t rows,
|
||||
@ -1240,7 +1240,7 @@ protected:
|
||||
const String & path);
|
||||
|
||||
public:
|
||||
/// Шаблоны, инстанцирующиеся путём динамической компиляции кода - см. SpecializedAggregator.h
|
||||
/// Templates that are instantiated by dynamic code compilation - see SpecializedAggregator.h
|
||||
|
||||
template <typename Method, typename AggregateFunctionsList>
|
||||
void executeSpecialized(
|
||||
@ -1274,14 +1274,14 @@ public:
|
||||
Arena * arena) const;
|
||||
|
||||
protected:
|
||||
/// Слить данные из хэш-таблицы src в dst.
|
||||
/// Merge data from hash table `src` into `dst`.
|
||||
template <typename Method, typename Table>
|
||||
void mergeDataImpl(
|
||||
Table & table_dst,
|
||||
Table & table_src,
|
||||
Arena * arena) const;
|
||||
|
||||
/// Слить данные из хэш-таблицы src в dst, но только для ключей, которые уже есть в dst. В остальных случаях, слить данные в overflows.
|
||||
/// Merge data from hash table `src` into `dst`, but only for keys that already exist in dst. In other cases, merge the data into `overflows`.
|
||||
template <typename Method, typename Table>
|
||||
void mergeDataNoMoreKeysImpl(
|
||||
Table & table_dst,
|
||||
@ -1289,7 +1289,7 @@ protected:
|
||||
Table & table_src,
|
||||
Arena * arena) const;
|
||||
|
||||
/// То же самое, но игнорирует остальные ключи.
|
||||
/// Same, but ignores the rest of the keys.
|
||||
template <typename Method, typename Table>
|
||||
void mergeDataOnlyExistingKeysImpl(
|
||||
Table & table_dst,
|
||||
@ -1400,17 +1400,17 @@ protected:
|
||||
AggregatedDataVariants & result) const;
|
||||
|
||||
|
||||
/** Проверяет ограничения на максимальное количество ключей для агрегации.
|
||||
* Если оно превышено, то, в зависимости от group_by_overflow_mode, либо
|
||||
* - кидает исключение;
|
||||
* - возвращает false, что говорит о том, что выполнение нужно прервать;
|
||||
* - выставляет переменную no_more_keys в true.
|
||||
/** Checks constraints on the maximum number of keys for aggregation.
|
||||
* If it is exceeded, then, depending on the group_by_overflow_mode, either
|
||||
* - throws an exception;
|
||||
* - returns false, which means that execution must be aborted;
|
||||
* - sets the variable no_more_keys to true.
|
||||
*/
|
||||
bool checkLimits(size_t result_size, bool & no_more_keys) const;
|
||||
};
|
||||
|
||||
|
||||
/** Достать вариант агрегации по его типу. */
|
||||
/** Get the aggregation variant by its type. */
|
||||
template <typename Method> Method & getDataVariant(AggregatedDataVariants & variants);
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
|
@ -9,23 +9,23 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Cluster содержит пулы соединений до каждого из узлов
|
||||
/// С локальными узлами соединение не устанавливается, а выполяется запрос напрямую.
|
||||
/// Поэтому храним только количество локальных узлов
|
||||
/// В конфиге кластер включает в себя узлы <node> или <shard>
|
||||
/// Cluster contains connection pools to each node
|
||||
/// With the local nodes, the connection is not established, but the request is executed directly.
|
||||
/// Therefore we store only the number of local nodes
|
||||
/// In the config, the cluster includes nodes <node> or <shard>
|
||||
class Cluster
|
||||
{
|
||||
public:
|
||||
Cluster(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & cluster_name);
|
||||
|
||||
/// Построить кластер по именам шардов и реплик. Локальные обрабатываются так же как удаленные.
|
||||
/// Construct a cluster by the names of shards and replicas. Local are treated as well as remote ones.
|
||||
Cluster(const Settings & settings, const std::vector<std::vector<String>> & names,
|
||||
const String & username, const String & password);
|
||||
|
||||
Cluster(const Cluster &) = delete;
|
||||
Cluster & operator=(const Cluster &) = delete;
|
||||
|
||||
/// используеться для выставления ограничения на размер таймаута
|
||||
/// is used to set a limit on the size of the timeout
|
||||
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
|
||||
|
||||
public:
|
||||
@ -73,7 +73,7 @@ public:
|
||||
public:
|
||||
/// contains names of directories for asynchronous write to StorageDistributed
|
||||
std::vector<std::string> dir_names;
|
||||
UInt32 shard_num; /// Номер шарда, начиная с 1.
|
||||
UInt32 shard_num; /// Shard number, starting with 1.
|
||||
int weight;
|
||||
Addresses local_addresses;
|
||||
ConnectionPoolWithFailoverPtr pool;
|
||||
@ -93,17 +93,17 @@ public:
|
||||
return shards_info.front();
|
||||
}
|
||||
|
||||
/// Количество удалённых шардов.
|
||||
/// The number of remote shards.
|
||||
size_t getRemoteShardCount() const { return remote_shard_count; }
|
||||
|
||||
/// Количество узлов clickhouse сервера, расположенных локально
|
||||
/// к локальным узлам обращаемся напрямую.
|
||||
/// The number of clickhouse nodes located locally
|
||||
/// we access the local nodes directly.
|
||||
size_t getLocalShardCount() const { return local_shard_count; }
|
||||
|
||||
/// Количество всех шардов.
|
||||
/// The number of all shards.
|
||||
size_t getShardCount() const { return shards_info.size(); }
|
||||
|
||||
/// Получить подкластер, состоящий из одного шарда - index по счёту (с нуля) шарда данного кластера.
|
||||
/// Get a subcluster consisting of one shard - index by count (from 0) of the shard of this cluster.
|
||||
std::unique_ptr<Cluster> getClusterWithSingleShard(size_t index) const;
|
||||
|
||||
private:
|
||||
@ -121,21 +121,21 @@ private:
|
||||
/// on tables that have the distributed engine.
|
||||
void calculateHashOfAddresses();
|
||||
|
||||
/// Для реализации getClusterWithSingleShard.
|
||||
/// For getClusterWithSingleShard implementation.
|
||||
Cluster(const Cluster & from, size_t index);
|
||||
|
||||
String hash_of_addresses;
|
||||
/// Описание шардов кластера.
|
||||
/// Description of the cluster shards.
|
||||
ShardsInfo shards_info;
|
||||
/// Любой удалённый шард.
|
||||
/// Any remote shard.
|
||||
ShardInfo * any_remote_shard_info = nullptr;
|
||||
|
||||
/// Непустым является либо addresses, либо addresses_with_failover.
|
||||
/// Размер и порядок элементов в соответствующем массиве соответствует shards_info.
|
||||
/// Non-empty is either addresses or addresses_with_failover.
|
||||
/// The size and order of the elements in the corresponding array corresponds to shards_info.
|
||||
|
||||
/// Массив шардов. Каждый шард - адреса одного сервера.
|
||||
/// An array of shards. Each shard is the address of one server.
|
||||
Addresses addresses;
|
||||
/// Массив шардов. Для каждого шарда - массив адресов реплик (серверов, считающихся идентичными).
|
||||
/// An array of shards. For each shard, an array of replica addresses (servers that are considered identical).
|
||||
AddressesWithFailover addresses_with_failover;
|
||||
|
||||
size_t remote_shard_count = 0;
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <ext/unlock_guard.hpp>
|
||||
#include <ext/unlock_guard.h>
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/ShellCommand.h>
|
||||
|
@ -20,7 +20,7 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Позволяет открыть динамическую библиотеку и получить из неё указатель на функцию.
|
||||
/** Allows you to open a dynamic library and get a pointer to a function from it.
|
||||
*/
|
||||
class SharedLibrary : private boost::noncopyable
|
||||
{
|
||||
@ -58,17 +58,17 @@ private:
|
||||
using SharedLibraryPtr = std::shared_ptr<SharedLibrary>;
|
||||
|
||||
|
||||
/** Позволяет скомпилировать кусок кода, использующий заголовочные файлы сервера, в динамическую библиотеку.
|
||||
* Ведёт статистику вызовов, и инициирует компиляцию только на N-ый по счёту вызов для одного ключа.
|
||||
* Компиляция выполняется асинхронно, в отдельных потоках, если есть свободные потоки.
|
||||
* NOTE: Нет очистки устаревших и ненужных результатов.
|
||||
/** Lets you compile a piece of code that uses the server's header files into the dynamic library.
|
||||
* Conducts statistic of calls, and initiates compilation only on the N-th call for one key.
|
||||
* Compilation is performed asynchronously, in separate threads, if there are free threads.
|
||||
* NOTE: There is no cleaning of obsolete and unnecessary results.
|
||||
*/
|
||||
class Compiler
|
||||
{
|
||||
public:
|
||||
/** path - путь к директории с временными файлами - результатами компиляции.
|
||||
* Результаты компиляции сохраняются при перезапуске сервера,
|
||||
* но используют в качестве части ключа номер ревизии. То есть, устаревают при обновлении сервера.
|
||||
/** path - path to the directory with temporary files - the results of the compilation.
|
||||
* The compilation results are saved when the server is restarted,
|
||||
* but use the revision number as part of the key. That is, they become obsolete when the server is updated.
|
||||
*/
|
||||
Compiler(const std::string & path_, size_t threads);
|
||||
~Compiler();
|
||||
@ -78,13 +78,13 @@ public:
|
||||
using CodeGenerator = std::function<std::string()>;
|
||||
using ReadyCallback = std::function<void(SharedLibraryPtr&)>;
|
||||
|
||||
/** Увеличить счётчик для заданного ключа key на единицу.
|
||||
* Если результат компиляции уже есть (уже открыт, или есть файл с библиотекой),
|
||||
* то вернуть готовую SharedLibrary.
|
||||
* Иначе, если min_count_to_compile == 0, то инициировать компиляцию в том же потоке, дождаться её, и вернуть результат.
|
||||
* Иначе, если счётчик достиг min_count_to_compile,
|
||||
* инициировать компиляцию в отдельном потоке, если есть свободные потоки, и вернуть nullptr.
|
||||
* Иначе вернуть nullptr.
|
||||
/** Increase the counter for the given key `key` by one.
|
||||
* If the compilation result already exists (already open, or there is a file with the library),
|
||||
* then return ready SharedLibrary.
|
||||
* Otherwise, if min_count_to_compile == 0, then initiate the compilation in the same thread, wait for it, and return the result.
|
||||
* Otherwise, if the counter has reached min_count_to_compile,
|
||||
* initiate compilation in a separate thread, if there are free threads, and return nullptr.
|
||||
* Otherwise, return nullptr.
|
||||
*/
|
||||
SharedLibraryPtr getOrCount(
|
||||
const std::string & key,
|
||||
@ -101,13 +101,13 @@ private:
|
||||
const std::string path;
|
||||
ThreadPool pool;
|
||||
|
||||
/// Количество вызовов функции getOrCount.
|
||||
/// Number of calls to `getOrCount`.
|
||||
Counts counts;
|
||||
|
||||
/// Скомпилированные и открытые библиотеки. Или nullptr для библиотек в процессе компиляции.
|
||||
/// Compiled and open libraries. Or nullptr for libraries in the compilation process.
|
||||
Libraries libraries;
|
||||
|
||||
/// Скомпилированные файлы, оставшиеся от предыдущих запусков, но ещё не открытые.
|
||||
/// Compiled files remaining from previous runs, but not yet open.
|
||||
Files files;
|
||||
|
||||
std::mutex mutex;
|
||||
|
@ -34,6 +34,7 @@
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/InterserverIOHandler.h>
|
||||
#include <Interpreters/Compiler.h>
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -46,6 +47,7 @@
|
||||
|
||||
#include <Common/ConfigProcessor.h>
|
||||
#include <zkutil/ZooKeeper.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -105,7 +107,7 @@ struct ContextShared
|
||||
|
||||
String path; /// Path to the data directory, with a slash at the end.
|
||||
String tmp_path; /// The path to the temporary files that occur when processing the request.
|
||||
String flags_path; ///
|
||||
String flags_path; /// Path to the directory with some control flags for server maintenance.
|
||||
Databases databases; /// List of databases and tables in them.
|
||||
TableFunctionFactory table_function_factory; /// Table functions.
|
||||
FormatFactory format_factory; /// Formats.
|
||||
@ -123,16 +125,17 @@ struct ContextShared
|
||||
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
|
||||
BackgroundProcessingPoolPtr background_pool; /// The thread pool for the background work performed by the tables.
|
||||
ReshardingWorkerPtr resharding_worker;
|
||||
Macros macros; /// Substitutions extracted from config.
|
||||
Macros macros; /// Substitutions from config. Can be used for parameters of ReplicatedMergeTree.
|
||||
std::unique_ptr<Compiler> compiler; /// Used for dynamic compilation of queries' parts if it necessary.
|
||||
std::unique_ptr<QueryLog> query_log; /// Used to log queries.
|
||||
std::shared_ptr<PartLog> part_log; /// Used to log operations with parts
|
||||
/// Rules for selecting the compression method, depending on the size of the part.
|
||||
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector;
|
||||
std::unique_ptr<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
|
||||
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
|
||||
|
||||
class SessionKeyHash {
|
||||
/// Named sessions. The user could specify session identifier to reuse settings and temporary tables in subsequent requests.
|
||||
|
||||
class SessionKeyHash
|
||||
{
|
||||
public:
|
||||
size_t operator()(const Context::SessionKey & key) const
|
||||
{
|
||||
@ -196,9 +199,6 @@ struct ContextShared
|
||||
return;
|
||||
shutdown_called = true;
|
||||
|
||||
query_log.reset();
|
||||
part_log.reset();
|
||||
|
||||
/** At this point, some tables may have threads that block our mutex.
|
||||
* To complete them correctly, we will copy the current list of tables,
|
||||
* and ask them all to finish their work.
|
||||
@ -224,12 +224,24 @@ struct ContextShared
|
||||
|
||||
|
||||
Context::Context()
|
||||
: shared(new ContextShared),
|
||||
quota(new QuotaForIntervals)
|
||||
: shared(std::make_shared<ContextShared>()),
|
||||
quota(std::make_shared<QuotaForIntervals>()),
|
||||
system_logs(std::make_shared<SystemLogs>())
|
||||
{
|
||||
}
|
||||
|
||||
Context::~Context() = default;
|
||||
Context::~Context()
|
||||
{
|
||||
try
|
||||
{
|
||||
/// Destroy system logs while at least one Context is alive
|
||||
system_logs.reset();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
const TableFunctionFactory & Context::getTableFunctionFactory() const { return shared->table_function_factory; }
|
||||
@ -272,14 +284,14 @@ Context::SessionKey Context::getSessionKey(const String & session_id) const
|
||||
}
|
||||
|
||||
|
||||
void Context::scheduleClose(const Context::SessionKey & key, std::chrono::steady_clock::duration timeout)
|
||||
void Context::scheduleCloseSession(const Context::SessionKey & key, std::chrono::steady_clock::duration timeout)
|
||||
{
|
||||
const UInt64 close_index = timeout / shared->close_interval + 1;
|
||||
const auto new_close_cycle = shared->close_cycle + close_index;
|
||||
|
||||
if (close_cycle != new_close_cycle)
|
||||
if (session_close_cycle != new_close_cycle)
|
||||
{
|
||||
close_cycle = new_close_cycle;
|
||||
session_close_cycle = new_close_cycle;
|
||||
if (shared->close_times.size() < close_index + 1)
|
||||
shared->close_times.resize(close_index + 1);
|
||||
shared->close_times[close_index].emplace_back(key);
|
||||
@ -301,7 +313,7 @@ std::shared_ptr<Context> Context::acquireSession(const String & session_id, std:
|
||||
|
||||
auto new_session = std::make_shared<Context>(*global_context);
|
||||
|
||||
new_session->scheduleClose(key, timeout);
|
||||
new_session->scheduleCloseSession(key, timeout);
|
||||
|
||||
it = shared->sessions.insert(std::make_pair(key, std::move(new_session))).first;
|
||||
}
|
||||
@ -312,9 +324,9 @@ std::shared_ptr<Context> Context::acquireSession(const String & session_id, std:
|
||||
|
||||
const auto & session = it->second;
|
||||
|
||||
if (session->used)
|
||||
if (session->session_is_used)
|
||||
throw Exception("Session is locked by a concurrent client.", ErrorCodes::SESSION_IS_LOCKED);
|
||||
session->used = true;
|
||||
session->session_is_used = true;
|
||||
|
||||
session->client_info = client_info;
|
||||
|
||||
@ -326,9 +338,8 @@ void Context::releaseSession(const String & session_id, std::chrono::steady_cloc
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
used = false;
|
||||
|
||||
scheduleClose(getSessionKey(session_id), timeout);
|
||||
session_is_used = false;
|
||||
scheduleCloseSession(getSessionKey(session_id), timeout);
|
||||
}
|
||||
|
||||
|
||||
@ -343,7 +354,7 @@ std::chrono::steady_clock::duration Context::closeSessions() const
|
||||
|
||||
const auto current_cycle = shared->close_cycle;
|
||||
|
||||
++(shared->close_cycle);
|
||||
++shared->close_cycle;
|
||||
shared->close_cycle_time = now + shared->close_interval;
|
||||
|
||||
if (shared->close_times.empty())
|
||||
@ -355,10 +366,10 @@ std::chrono::steady_clock::duration Context::closeSessions() const
|
||||
{
|
||||
const auto session = shared->sessions.find(key);
|
||||
|
||||
if (session != shared->sessions.end() && session->second->close_cycle <= current_cycle)
|
||||
if (session != shared->sessions.end() && session->second->session_close_cycle <= current_cycle)
|
||||
{
|
||||
if (session->second->used)
|
||||
session->second->scheduleClose(key, std::chrono::seconds(0));
|
||||
if (session->second->session_is_used)
|
||||
session->second->scheduleCloseSession(key, std::chrono::seconds(0));
|
||||
else
|
||||
shared->sessions.erase(session);
|
||||
}
|
||||
@ -1206,10 +1217,13 @@ QueryLog & Context::getQueryLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (!shared->query_log)
|
||||
if (!system_logs)
|
||||
throw Exception("Query log have been already shutdown", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!system_logs->query_log)
|
||||
{
|
||||
if (shared->shutdown_called)
|
||||
throw Exception("Will not get query_log because shutdown was called", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Logical error: query log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!global_context)
|
||||
throw Exception("Logical error: no global context for query log", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -1221,15 +1235,15 @@ QueryLog & Context::getQueryLog()
|
||||
size_t flush_interval_milliseconds = config.getUInt64(
|
||||
"query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS);
|
||||
|
||||
shared->query_log = std::make_unique<QueryLog>(
|
||||
system_logs->query_log = std::make_unique<QueryLog>(
|
||||
*global_context, database, table, "MergeTree(event_date, event_time, 1024)", flush_interval_milliseconds);
|
||||
}
|
||||
|
||||
return *shared->query_log;
|
||||
return *system_logs->query_log;
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<PartLog> Context::getPartLog()
|
||||
PartLog * Context::getPartLog(const String & database, const String & table)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -1237,23 +1251,33 @@ std::shared_ptr<PartLog> Context::getPartLog()
|
||||
if (!config.has("part_log"))
|
||||
return nullptr;
|
||||
|
||||
if (!shared->part_log)
|
||||
/// System logs are shutdown
|
||||
if (!system_logs)
|
||||
return nullptr;
|
||||
|
||||
String part_log_database = config.getString("part_log.database", "system");
|
||||
String part_log_table = config.getString("part_log.table", "part_log");
|
||||
|
||||
/// Will not log system.part_log itself.
|
||||
/// It doesn't make sense and not allow to destruct PartLog correctly due to infinite logging and flushing
|
||||
if (database == part_log_database && table == part_log_table)
|
||||
return nullptr;
|
||||
|
||||
if (!system_logs->part_log)
|
||||
{
|
||||
if (shared->shutdown_called)
|
||||
throw Exception("Will not get part_log because shutdown was called", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Logical error: part log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!global_context)
|
||||
throw Exception("Logical error: no global context for part log", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String database = config.getString("part_log.database", "system");
|
||||
String table = config.getString("part_log.table", "part_log");
|
||||
size_t flush_interval_milliseconds = config.getUInt64(
|
||||
"part_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS);
|
||||
shared->part_log = std::make_unique<PartLog>(
|
||||
*global_context, database, table, "MergeTree(event_date, event_time, 1024)", flush_interval_milliseconds);
|
||||
system_logs->part_log = std::make_unique<PartLog>(*global_context, part_log_database, part_log_table,
|
||||
"MergeTree(event_date, event_time, 1024)", flush_interval_milliseconds);
|
||||
}
|
||||
|
||||
return shared->part_log;
|
||||
return system_logs->part_log.get();
|
||||
}
|
||||
|
||||
|
||||
@ -1360,6 +1384,7 @@ time_t Context::getUptimeSeconds() const
|
||||
|
||||
void Context::shutdown()
|
||||
{
|
||||
system_logs.reset();
|
||||
shared->shutdown();
|
||||
}
|
||||
|
||||
|
@ -65,21 +65,23 @@ class IBlockOutputStream;
|
||||
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
||||
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
|
||||
class Block;
|
||||
struct SystemLogs;
|
||||
using SystemLogsPtr = std::shared_ptr<SystemLogs>;
|
||||
|
||||
|
||||
/// (имя базы данных, имя таблицы)
|
||||
/// (database name, table name)
|
||||
using DatabaseAndTableName = std::pair<String, String>;
|
||||
|
||||
/// Таблица -> множество таблиц-представлений, которые деляют SELECT из неё.
|
||||
/// Table -> set of table-views that make SELECT from it.
|
||||
using ViewDependencies = std::map<DatabaseAndTableName, std::set<DatabaseAndTableName>>;
|
||||
using Dependencies = std::vector<DatabaseAndTableName>;
|
||||
|
||||
|
||||
/** Набор известных объектов, которые могут быть использованы в запросе.
|
||||
* Состоит из разделяемой части (всегда общей для всех сессий и запросов)
|
||||
* и копируемой части (которая может быть своей для каждой сессии или запроса).
|
||||
/** A set of known objects that can be used in the query.
|
||||
* Consists of a shared part (always common to all sessions and queries)
|
||||
* and copied part (which can be its own for each session or query).
|
||||
*
|
||||
* Всё инкапсулировано для всяких проверок и блокировок.
|
||||
* Everything is encapsulated for all sorts of checks and locks.
|
||||
*/
|
||||
class Context
|
||||
{
|
||||
@ -101,9 +103,10 @@ private:
|
||||
Tables external_tables; /// Temporary tables.
|
||||
Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this.
|
||||
Context * global_context = nullptr; /// Global context or nullptr. Could be equal to this.
|
||||
SystemLogsPtr system_logs; /// Used to log queries and operations on parts
|
||||
|
||||
UInt64 close_cycle = 0;
|
||||
bool used = false;
|
||||
UInt64 session_close_cycle = 0;
|
||||
bool session_is_used = false;
|
||||
|
||||
using DatabasePtr = std::shared_ptr<IDatabase>;
|
||||
using Databases = std::map<String, std::shared_ptr<IDatabase>>;
|
||||
@ -121,9 +124,9 @@ public:
|
||||
|
||||
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
|
||||
|
||||
/** Забрать список пользователей, квот и профилей настроек из этого конфига.
|
||||
* Список пользователей полностью заменяется.
|
||||
* Накопленные значения у квоты не сбрасываются, если квота не удалена.
|
||||
/** Take the list of users, quotas and configuration profiles from this config.
|
||||
* The list of users is completely replaced.
|
||||
* The accumulated quota values are not reset if the quota is not deleted.
|
||||
*/
|
||||
void setUsersConfig(const ConfigurationPtr & config);
|
||||
|
||||
@ -142,14 +145,14 @@ public:
|
||||
void removeDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where);
|
||||
Dependencies getDependencies(const String & database_name, const String & table_name) const;
|
||||
|
||||
/// Проверка существования таблицы/БД. database может быть пустой - в этом случае используется текущая БД.
|
||||
/// Checking the existence of the table/database. Database can be empty - in this case the current database is used.
|
||||
bool isTableExist(const String & database_name, const String & table_name) const;
|
||||
bool isDatabaseExist(const String & database_name) const;
|
||||
void assertTableExists(const String & database_name, const String & table_name) const;
|
||||
|
||||
/** Параметр check_database_access_rights существует, чтобы не проверить повторно права доступа к БД,
|
||||
* когда assertTableDoesnExist или assertDatabaseExists вызывается внутри другой функции, которая уже
|
||||
* сделала эту проверку.
|
||||
/** The parameter check_database_access_rights exists to not check the permissions of the database again,
|
||||
* when assertTableDoesntExist or assertDatabaseExists is called inside another function that already
|
||||
* made this check.
|
||||
*/
|
||||
void assertTableDoesntExist(const String & database_name, const String & table_name, bool check_database_acccess_rights = true) const;
|
||||
void assertDatabaseExists(const String & database_name, bool check_database_acccess_rights = true) const;
|
||||
@ -165,10 +168,10 @@ public:
|
||||
void addDatabase(const String & database_name, const DatabasePtr & database);
|
||||
DatabasePtr detachDatabase(const String & database_name);
|
||||
|
||||
/// Получить объект, который защищает таблицу от одновременного выполнения нескольких DDL операций.
|
||||
/// Если такой объект уже есть - кидается исключение.
|
||||
/// Get an object that protects the table from concurrently executing multiple DDL operations.
|
||||
/// If such an object already exists, an exception is thrown.
|
||||
std::unique_ptr<DDLGuard> getDDLGuard(const String & database, const String & table, const String & message) const;
|
||||
/// Если таблица уже есть - возвращается nullptr, иначе создаётся guard.
|
||||
/// If the table already exists, it returns nullptr, otherwise guard is created.
|
||||
std::unique_ptr<DDLGuard> getDDLGuardIfTableDoesntExist(const String & database, const String & table, const String & message) const;
|
||||
|
||||
String getCurrentDatabase() const;
|
||||
@ -176,7 +179,7 @@ public:
|
||||
void setCurrentDatabase(const String & name);
|
||||
void setCurrentQueryId(const String & query_id);
|
||||
|
||||
String getDefaultFormat() const; /// Если default_format не задан - возвращается некоторый глобальный формат по-умолчанию.
|
||||
String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned.
|
||||
void setDefaultFormat(const String & name);
|
||||
|
||||
const Macros & getMacros() const;
|
||||
@ -187,10 +190,10 @@ public:
|
||||
|
||||
Limits getLimits() const;
|
||||
|
||||
/// Установить настройку по имени.
|
||||
/// Set a setting by name.
|
||||
void setSetting(const String & name, const Field & value);
|
||||
|
||||
/// Установить настройку по имени. Прочитать значение в текстовом виде из строки (например, из конфига, или из параметра URL).
|
||||
/// Set a setting by name. Read the value in text form from a string (for example, from a config, or from a URL parameter).
|
||||
void setSetting(const String & name, const std::string & value);
|
||||
|
||||
const TableFunctionFactory & getTableFunctionFactory() const;
|
||||
@ -199,19 +202,19 @@ public:
|
||||
void tryCreateEmbeddedDictionaries() const;
|
||||
void tryCreateExternalDictionaries() const;
|
||||
|
||||
/// Форматы ввода-вывода.
|
||||
/// I/O formats.
|
||||
BlockInputStreamPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, size_t max_block_size) const;
|
||||
BlockOutputStreamPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const;
|
||||
|
||||
InterserverIOHandler & getInterserverIOHandler();
|
||||
|
||||
/// Как другие серверы могут обратиться к этому для скачивания реплицируемых данных.
|
||||
/// How other servers can access this for downloading replicated data.
|
||||
void setInterserverIOAddress(const String & host, UInt16 port);
|
||||
std::pair<String, UInt16> getInterserverIOAddress() const;
|
||||
/// Порт, который сервер слушает для выполнения SQL-запросов.
|
||||
/// The port that the server listens for executing SQL queries.
|
||||
UInt16 getTCPPort() const;
|
||||
|
||||
/// Получить запрос на CREATE таблицы.
|
||||
/// Get query for the CREATE table.
|
||||
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;
|
||||
|
||||
const DatabasePtr getDatabase(const String & database_name) const;
|
||||
@ -222,12 +225,11 @@ public:
|
||||
const Databases getDatabases() const;
|
||||
Databases getDatabases();
|
||||
|
||||
using SessionKey = std::pair<String, String>;
|
||||
|
||||
std::shared_ptr<Context> acquireSession(const String & session_id, std::chrono::steady_clock::duration timeout, bool session_check) const;
|
||||
void releaseSession(const String & session_id, std::chrono::steady_clock::duration timeout);
|
||||
std::chrono::steady_clock::duration closeSessions() const;
|
||||
|
||||
/// Close sessions, that has been expired. Returns how long to wait for next session to be expired, if no new sessions will be added.
|
||||
std::chrono::steady_clock::duration closeSessions() const;
|
||||
|
||||
/// For methods below you may need to acquire a lock by yourself.
|
||||
std::unique_lock<Poco::Mutex> getLock() const;
|
||||
@ -246,32 +248,32 @@ public:
|
||||
|
||||
|
||||
void setProgressCallback(ProgressCallback callback);
|
||||
/// Используется в InterpreterSelectQuery, чтобы передать его в IProfilingBlockInputStream.
|
||||
/// Used in InterpreterSelectQuery to pass it to the IProfilingBlockInputStream.
|
||||
ProgressCallback getProgressCallback() const;
|
||||
|
||||
/** Устанавливается в executeQuery и InterpreterSelectQuery. Затем используется в IProfilingBlockInputStream,
|
||||
* чтобы обновлять и контролировать информацию об общем количестве потраченных на запрос ресурсов.
|
||||
/** Set in executeQuery and InterpreterSelectQuery. Then it is used in IProfilingBlockInputStream,
|
||||
* to update and monitor information about the total number of resources spent for the query.
|
||||
*/
|
||||
void setProcessListElement(ProcessListElement * elem);
|
||||
/// Может вернуть nullptr, если запрос не был вставлен в ProcessList.
|
||||
/// Can return nullptr if the query was not inserted into the ProcessList.
|
||||
ProcessListElement * getProcessListElement();
|
||||
|
||||
/// Список всех запросов.
|
||||
/// List all queries.
|
||||
ProcessList & getProcessList();
|
||||
const ProcessList & getProcessList() const;
|
||||
|
||||
MergeList & getMergeList();
|
||||
const MergeList & getMergeList() const;
|
||||
|
||||
/// Создать кэш разжатых блоков указанного размера. Это можно сделать только один раз.
|
||||
/// Create a cache of uncompressed blocks of specified size. This can be done only once.
|
||||
void setUncompressedCache(size_t max_size_in_bytes);
|
||||
std::shared_ptr<UncompressedCache> getUncompressedCache() const;
|
||||
|
||||
void setZooKeeper(std::shared_ptr<zkutil::ZooKeeper> zookeeper);
|
||||
/// Если в момент вызова текущая сессия просрочена, синхронно создает и возвращает новую вызовом startNewSession().
|
||||
/// If the current session is expired at the time of the call, synchronously creates and returns a new session with the startNewSession() call.
|
||||
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
|
||||
|
||||
/// Создать кэш засечек указанного размера. Это можно сделать только один раз.
|
||||
/// Create a cache of marks of specified size. This can be done only once.
|
||||
void setMarkCache(size_t cache_size_in_bytes);
|
||||
std::shared_ptr<MarkCache> getMarkCache() const;
|
||||
|
||||
@ -280,11 +282,11 @@ public:
|
||||
void setReshardingWorker(std::shared_ptr<ReshardingWorker> resharding_worker);
|
||||
ReshardingWorker & getReshardingWorker();
|
||||
|
||||
/** Очистить кэши разжатых блоков и засечек.
|
||||
* Обычно это делается при переименовании таблиц, изменении типа столбцов, удалении таблицы.
|
||||
* - так как кэши привязаны к именам файлов, и становятся некорректными.
|
||||
* (при удалении таблицы - нужно, так как на её месте может появиться другая)
|
||||
* const - потому что изменение кэша не считается существенным.
|
||||
/** Clear the caches of the uncompressed blocks and marks.
|
||||
* This is usually done when renaming tables, changing the type of columns, deleting a table.
|
||||
* - since caches are linked to file names, and become incorrect.
|
||||
* (when deleting a table - it is necessary, since in its place another can appear)
|
||||
* const - because the change in the cache is not considered significant.
|
||||
*/
|
||||
void resetCaches() const;
|
||||
|
||||
@ -295,17 +297,20 @@ public:
|
||||
|
||||
Compiler & getCompiler();
|
||||
QueryLog & getQueryLog();
|
||||
std::shared_ptr<PartLog> getPartLog();
|
||||
|
||||
/// Returns an object used to log opertaions with parts if it possible.
|
||||
/// Provide table name to make required cheks.
|
||||
PartLog * getPartLog(const String & database, const String & table);
|
||||
const MergeTreeSettings & getMergeTreeSettings();
|
||||
|
||||
/// Prevents DROP TABLE if its size is greater than max_size (50GB by default, max_size=0 turn off this check)
|
||||
void setMaxTableSizeToDrop(size_t max_size);
|
||||
void checkTableCanBeDropped(const String & database, const String & table, size_t table_size);
|
||||
|
||||
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
|
||||
/// Lets you select the compression method according to the conditions described in the configuration file.
|
||||
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;
|
||||
|
||||
/// Получить аптайм сервера в секундах.
|
||||
/// Get the server uptime in seconds.
|
||||
time_t getUptimeSeconds() const;
|
||||
|
||||
void shutdown();
|
||||
@ -324,10 +329,13 @@ public:
|
||||
String getDefaultProfileName() const;
|
||||
void setDefaultProfileName(const String & name);
|
||||
|
||||
/// User name and session identifier. Named sessions are local to users.
|
||||
using SessionKey = std::pair<String, String>;
|
||||
|
||||
private:
|
||||
/** Проверить, имеет ли текущий клиент доступ к заданной базе данных.
|
||||
* Если доступ запрещён, кинуть исключение.
|
||||
* NOTE: Этот метод надо всегда вызывать при захваченном мьютексе shared->mutex.
|
||||
/** Check if the current client has access to the specified database.
|
||||
* If access is denied, throw an exception.
|
||||
* NOTE: This method should always be called when the `shared->mutex` mutex is acquired.
|
||||
*/
|
||||
void checkDatabaseAccessRights(const std::string & database_name) const;
|
||||
|
||||
@ -337,7 +345,9 @@ private:
|
||||
StoragePtr getTableImpl(const String & database_name, const String & table_name, Exception * exception) const;
|
||||
|
||||
SessionKey getSessionKey(const String & session_id) const;
|
||||
void scheduleClose(const SessionKey & key, std::chrono::steady_clock::duration timeout);
|
||||
|
||||
/// Session will be closed after specified timeout.
|
||||
void scheduleCloseSession(const SessionKey & key, std::chrono::steady_clock::duration timeout);
|
||||
};
|
||||
|
||||
|
||||
|
@ -32,7 +32,7 @@ class IBlockInputStream;
|
||||
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
||||
|
||||
|
||||
/** Действие над блоком.
|
||||
/** Action on the block.
|
||||
*/
|
||||
struct ExpressionAction
|
||||
{
|
||||
@ -45,16 +45,16 @@ public:
|
||||
|
||||
APPLY_FUNCTION,
|
||||
|
||||
/** Заменяет указанные столбцы с массивами на столбцы с элементами.
|
||||
* Размножает значения в остальных столбцах по количеству элементов в массивах.
|
||||
* Массивы должны быть параллельными (иметь одинаковые длины).
|
||||
/** Replaces the specified columns with arrays into columns with elements.
|
||||
* Duplicates the values in the remaining columns by the number of elements in the arrays.
|
||||
* Arrays must be parallel (have the same lengths).
|
||||
*/
|
||||
ARRAY_JOIN,
|
||||
|
||||
/// INNER|LEFT JOIN.
|
||||
JOIN,
|
||||
|
||||
/// Переупорядочить и переименовать столбцы, удалить лишние. Допускаются одинаковые имена столбцов в результате.
|
||||
/// Reorder and rename the columns, delete the extra ones. The same column names are allowed in the result.
|
||||
PROJECT,
|
||||
};
|
||||
|
||||
@ -84,7 +84,7 @@ public:
|
||||
/// For PROJECT.
|
||||
NamesWithAliases projection;
|
||||
|
||||
/// Если result_name_ == "", в качестве имени используется "имя_функции(аргументы через запятую)".
|
||||
/// If result_name_ == "", as name "function_name(arguments separated by commas) is used".
|
||||
static ExpressionAction applyFunction(
|
||||
const FunctionPtr & function_, const std::vector<std::string> & argument_names_, std::string result_name_ = "");
|
||||
|
||||
@ -96,8 +96,8 @@ public:
|
||||
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context);
|
||||
static ExpressionAction ordinaryJoin(std::shared_ptr<const Join> join_, const NamesAndTypesList & columns_added_by_join_);
|
||||
|
||||
/// Какие столбцы нужны, чтобы выполнить это действие.
|
||||
/// Если этот Action еще не добавлен в ExpressionActions, возвращаемый список может быть неполным, потому что не учтены prerequisites.
|
||||
/// Which columns necessary to perform this action.
|
||||
/// If this `Action` is not already added to `ExpressionActions`, the returned list may be incomplete, because `prerequisites` are not taken into account.
|
||||
Names getNeededColumns() const;
|
||||
|
||||
std::string toString() const;
|
||||
@ -112,7 +112,7 @@ private:
|
||||
};
|
||||
|
||||
|
||||
/** Содержит последовательность действий над блоком.
|
||||
/** Contains a sequence of actions on the block.
|
||||
*/
|
||||
class ExpressionActions
|
||||
{
|
||||
@ -126,7 +126,7 @@ public:
|
||||
sample_block.insert(ColumnWithTypeAndName(nullptr, input_elem.type, input_elem.name));
|
||||
}
|
||||
|
||||
/// Для константных столбцов в input_columns_ могут содержаться сами столбцы.
|
||||
/// For constant columns the columns themselves can be contained in `input_columns_`.
|
||||
ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Settings & settings_)
|
||||
: settings(settings_)
|
||||
{
|
||||
@ -137,40 +137,40 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
/// Добавить входной столбец.
|
||||
/// Название столбца не должно совпадать с названиями промежуточных столбцов, возникающих при вычислении выражения.
|
||||
/// В выражении не должно быть действий PROJECT.
|
||||
/// Add the input column.
|
||||
/// The name of the column must not match the names of the intermediate columns that occur when evaluating the expression.
|
||||
/// The expression must not have any PROJECT actions.
|
||||
void addInput(const ColumnWithTypeAndName & column);
|
||||
void addInput(const NameAndTypePair & column);
|
||||
|
||||
void add(const ExpressionAction & action);
|
||||
|
||||
/// Кладет в out_new_columns названия новых столбцов
|
||||
/// (образовавшихся в результате добавляемого действия и его rerequisites).
|
||||
/// Adds new column names to out_new_columns
|
||||
/// (formed as a result of the added action and its prerequisites).
|
||||
void add(const ExpressionAction & action, Names & out_new_columns);
|
||||
|
||||
/// Добавляет в начало удаление всех лишних столбцов.
|
||||
/// Adds to the beginning the removal of all extra columns.
|
||||
void prependProjectInput();
|
||||
|
||||
/// Добавить в начало указанное действие типа ARRAY JOIN. Поменять соответствующие входные типы на массивы.
|
||||
/// Если в списке ARRAY JOIN есть неизвестные столбцы, взять их типы из sample_block, а сразу после ARRAY JOIN удалить.
|
||||
/// Add the specified ARRAY JOIN action to the beginning. Change the appropriate input types to arrays.
|
||||
/// If there are unknown columns in the ARRAY JOIN list, take their types from sample_block, and immediately after ARRAY JOIN remove them.
|
||||
void prependArrayJoin(const ExpressionAction & action, const Block & sample_block);
|
||||
|
||||
/// Если последнее действие - ARRAY JOIN, и оно не влияет на столбцы из required_columns, выбросить и вернуть его.
|
||||
/// Поменять соответствующие выходные типы на массивы.
|
||||
/// If the last action is ARRAY JOIN, and it does not affect the columns from required_columns, discard and return it.
|
||||
/// Change the corresponding output types to arrays.
|
||||
bool popUnusedArrayJoin(const Names & required_columns, ExpressionAction & out_action);
|
||||
|
||||
/// - Добавляет действия для удаления всех столбцов, кроме указанных.
|
||||
/// - Убирает неиспользуемые входные столбцы.
|
||||
/// - Может как-нибудь оптимизировать выражение.
|
||||
/// - Не переупорядочивает столбцы.
|
||||
/// - Не удаляет "неожиданные" столбцы (например, добавленные функциями).
|
||||
/// - Если output_columns пуст, оставляет один произвольный столбец (чтобы не потерялось количество строк в блоке).
|
||||
/// - Adds actions to delete all but the specified columns.
|
||||
/// - Removes unused input columns.
|
||||
/// - Can somehow optimize the expression.
|
||||
/// - Does not reorder the columns.
|
||||
/// - Does not remove "unexpected" columns (for example, added by functions).
|
||||
/// - If output_columns is empty, leaves one arbitrary column (so that the number of rows in the block is not lost).
|
||||
void finalize(const Names & output_columns);
|
||||
|
||||
const Actions & getActions() const { return actions; }
|
||||
|
||||
/// Получить список входных столбцов.
|
||||
/// Get a list of input columns.
|
||||
Names getRequiredColumns() const
|
||||
{
|
||||
Names names;
|
||||
@ -181,15 +181,15 @@ public:
|
||||
|
||||
const NamesAndTypesList & getRequiredColumnsWithTypes() const { return input_columns; }
|
||||
|
||||
/// Выполнить выражение над блоком. Блок должен содержать все столбцы , возвращаемые getRequiredColumns.
|
||||
/// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns.
|
||||
void execute(Block & block) const;
|
||||
|
||||
/** Выполнить выражение над блоком тотальных значений.
|
||||
* Почти не отличается от execute. Разница лишь при выполнении JOIN-а.
|
||||
/** Execute the expression on the block of total values.
|
||||
* Almost the same as `execute`. The difference is only when JOIN is executed.
|
||||
*/
|
||||
void executeOnTotals(Block & block) const;
|
||||
|
||||
/// Получить блок-образец, содержащий имена и типы столбцов результата.
|
||||
/// Obtain a sample block that contains the names and types of result columns.
|
||||
const Block & getSampleBlock() const { return sample_block; }
|
||||
|
||||
std::string getID() const;
|
||||
@ -208,27 +208,27 @@ private:
|
||||
|
||||
void checkLimits(Block & block) const;
|
||||
|
||||
/// Добавляет сначала все prerequisites, потом само действие.
|
||||
/// current_names - столбцы, prerequisites которых сейчас обрабатываются.
|
||||
/// Adds all `prerequisites` first, then the action itself.
|
||||
/// current_names - columns whose `prerequisites` are currently being processed.
|
||||
void addImpl(ExpressionAction action, NameSet & current_names, Names & new_names);
|
||||
|
||||
/// Попробовать что-нибудь улучшить, не меняя списки входных и выходных столбцов.
|
||||
/// Try to improve something without changing the lists of input and output columns.
|
||||
void optimize();
|
||||
/// Переместить все arrayJoin как можно ближе к концу.
|
||||
/// Move all arrayJoin as close as possible to the end.
|
||||
void optimizeArrayJoin();
|
||||
};
|
||||
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
|
||||
/** Последовательность преобразований над блоком.
|
||||
* Предполагается, что результат каждого шага подается на вход следующего шага.
|
||||
* Используется для выполнения некоторых частей запроса по отдельности.
|
||||
/** The sequence of transformations over the block.
|
||||
* It is assumed that the result of each step is fed to the input of the next step.
|
||||
* Used to execute parts of the query individually.
|
||||
*
|
||||
* Например, можно составить цепочку из двух шагов:
|
||||
* 1) вычислить выражение в секции WHERE,
|
||||
* 2) вычислить выражение в секции SELECT,
|
||||
* и между двумя шагами делать фильтрацию по значению в секции WHERE.
|
||||
* For example, you can create a chain of two steps:
|
||||
* 1) evaluate the expression in the WHERE clause,
|
||||
* 2) calculate the expression in the SELECT section,
|
||||
* and between the two steps do the filtering by value in the WHERE clause.
|
||||
*/
|
||||
struct ExpressionActionsChain
|
||||
{
|
||||
|
@ -52,7 +52,7 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#include <ext/range.hpp>
|
||||
#include <ext/range.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
|
||||
@ -534,6 +534,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t
|
||||
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>(sample.getColumnsList());
|
||||
|
||||
StoragePtr external_storage = StorageMemory::create(external_table_name, columns);
|
||||
external_storage->startup();
|
||||
|
||||
/** There are two ways to perform distributed GLOBAL subqueries.
|
||||
*
|
||||
|
@ -34,32 +34,32 @@ class ASTExpressionList;
|
||||
class ASTSelectQuery;
|
||||
|
||||
|
||||
/** Информация о том, что делать при выполнении подзапроса в секции [GLOBAL] IN/JOIN.
|
||||
/** Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section.
|
||||
*/
|
||||
struct SubqueryForSet
|
||||
{
|
||||
/// Источник - получен с помощью InterpreterSelectQuery подзапроса.
|
||||
/// The source is obtained using the InterpreterSelectQuery subquery.
|
||||
BlockInputStreamPtr source;
|
||||
Block source_sample;
|
||||
|
||||
/// Если задано - создать из результата Set.
|
||||
/// If set, create from Set result.
|
||||
SetPtr set;
|
||||
|
||||
/// Если задано - создать из результата Join.
|
||||
/// If set, create from Join result.
|
||||
JoinPtr join;
|
||||
|
||||
/// Если задано - положить результат в таблицу.
|
||||
/// Это - временная таблица для передачи на удалённые серверы при распределённой обработке запроса.
|
||||
/// If set, put the result into the table.
|
||||
/// This is a temporary table for transferring to remote servers for distributed query processing.
|
||||
StoragePtr table;
|
||||
};
|
||||
|
||||
/// ID подзапроса -> что с ним делать.
|
||||
/// ID of subquery -> what to do with it.
|
||||
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
|
||||
|
||||
|
||||
/** Превращает выражение из синтаксического дерева в последовательность действий для его выполнения.
|
||||
/** Transforms an expression from a syntax tree into a sequence of actions to execute it.
|
||||
*
|
||||
* NOTE: если ast - запрос SELECT из таблицы, структура этой таблицы не должна меняться во все время жизни ExpressionAnalyzer-а.
|
||||
* NOTE: if `ast` is a SELECT query from a table, the structure of this table should not change during the lifetime of ExpressionAnalyzer.
|
||||
*/
|
||||
class ExpressionAnalyzer : private boost::noncopyable
|
||||
{
|
||||
@ -75,20 +75,20 @@ public:
|
||||
size_t subquery_depth_ = 0,
|
||||
bool do_global_ = false);
|
||||
|
||||
/// Есть ли в выражении агрегатные функции или секция GROUP BY или HAVING.
|
||||
/// Does the expression have aggregate functions or a GROUP BY or HAVING section.
|
||||
bool hasAggregation() const { return has_aggregation; }
|
||||
|
||||
/// Получить список ключей агрегирования и описаний агрегатных функций, если в запросе есть GROUP BY.
|
||||
/// Get a list of aggregation keys and descriptions of aggregate functions if the query contains GROUP BY.
|
||||
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const;
|
||||
|
||||
/** Получить набор столбцов, которых достаточно прочитать из таблицы для вычисления выражения.
|
||||
* Не учитываются столбцы, добавляемые из другой таблицы путём JOIN-а.
|
||||
/** Get a set of columns that are enough to read from the table to evaluate the expression.
|
||||
* Columns added from another table by JOIN are not counted.
|
||||
*/
|
||||
Names getRequiredColumns();
|
||||
|
||||
/** Эти методы позволяют собрать цепочку преобразований над блоком, получающую значения в нужных секциях запроса.
|
||||
/** These methods allow you to build a chain of transformations over a block, that receives values in the desired sections of the query.
|
||||
*
|
||||
* Пример использования:
|
||||
* Example usage:
|
||||
* ExpressionActionsChain chain;
|
||||
* analyzer.appendWhere(chain);
|
||||
* chain.addStep();
|
||||
@ -96,48 +96,48 @@ public:
|
||||
* analyzer.appendOrderBy(chain);
|
||||
* chain.finalize();
|
||||
*
|
||||
* Если указано only_types = true, не выполняет подзапросы в соответствующих частях запроса. Полученные таким
|
||||
* образом действия не следует выполнять, они нужны только чтобы получить список столбцов с их типами.
|
||||
* If only_types = true set, does not execute subqueries in the relevant parts of the query. The actions got this way
|
||||
* shouldn't be executed, they are only needed to get a list of columns with their types.
|
||||
*/
|
||||
|
||||
/// До агрегации:
|
||||
/// Before aggregation:
|
||||
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
|
||||
|
||||
/// После агрегации:
|
||||
/// After aggregation:
|
||||
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendSelect(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendOrderBy(ExpressionActionsChain & chain, bool only_types);
|
||||
/// Удаляет все столбцы кроме выбираемых SELECT, упорядочивает оставшиеся столбцы и переименовывает их в алиасы.
|
||||
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
|
||||
void appendProjectResult(ExpressionActionsChain & chain, bool only_types) const;
|
||||
|
||||
/// Если ast не запрос SELECT, просто получает все действия для вычисления выражения.
|
||||
/// Если project_result, в выходном блоке останутся только вычисленные значения в нужном порядке, переименованные в алиасы.
|
||||
/// Иначе, из блока будут удаляться только временные столбцы.
|
||||
/// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression.
|
||||
/// If project_result, only the calculated values in the desired order, renamed to aliases, remain in the output block.
|
||||
/// Otherwise, only temporary columns will be deleted from the block.
|
||||
ExpressionActionsPtr getActions(bool project_result);
|
||||
|
||||
/// Действия, которые можно сделать над пустым блоком: добавление констант и применение функций, зависящих только от констант.
|
||||
/// Не выполняет подзапросы.
|
||||
/// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants.
|
||||
/// Does not execute subqueries.
|
||||
ExpressionActionsPtr getConstActions();
|
||||
|
||||
/** Множества, для создания которых нужно будет выполнить подзапрос.
|
||||
* Только множества, нужные для выполнения действий, возвращенных из уже вызванных append* или getActions.
|
||||
* То есть, нужно вызвать getSetsWithSubqueries после всех вызовов append* или getActions
|
||||
* и создать все возвращенные множества перед выполнением действий.
|
||||
/** Sets that require a subquery to be create.
|
||||
* Only the sets needed to perform actions returned from already executed `append*` or `getActions`.
|
||||
* That is, you need to call getSetsWithSubqueries after all calls of `append*` or `getActions`
|
||||
* and create all the returned sets before performing the actions.
|
||||
*/
|
||||
SubqueriesForSets getSubqueriesForSets() { return subqueries_for_sets; }
|
||||
|
||||
/** Таблицы, которые надо будет отправить на удалённые серверы при распределённой обработке запроса.
|
||||
/** Tables that will need to be sent to remote servers for distributed query processing.
|
||||
*/
|
||||
const Tables & getExternalTables() const { return external_tables; }
|
||||
|
||||
/// Если ast - запрос SELECT, получает имена (алиасы) и типы столбцов из секции SELECT.
|
||||
/// If ast is a SELECT query, it gets the aliases and column types from the SELECT section.
|
||||
Block getSelectSampleBlock();
|
||||
|
||||
/// Создаем какие сможем Set из секции IN для использования индекса по ним.
|
||||
/// Create Set-s that we can from IN section to use the index on them.
|
||||
void makeSetsForIndex();
|
||||
|
||||
private:
|
||||
@ -147,18 +147,18 @@ private:
|
||||
Settings settings;
|
||||
size_t subquery_depth;
|
||||
|
||||
/// Столбцы, которые упоминаются в выражении, но не были заданы в конструкторе.
|
||||
/// Columns that are mentioned in the expression, but were not specified in the constructor.
|
||||
NameSet unknown_required_columns;
|
||||
|
||||
/** Исходные столбцы.
|
||||
* Сначала сюда помещаются все доступные столбцы таблицы. Затем (при разборе запроса) удаляются неиспользуемые столбцы.
|
||||
/** Original columns.
|
||||
* First, all available columns of the table are placed here. Then (when parsing the query), unused columns are deleted.
|
||||
*/
|
||||
NamesAndTypesList columns;
|
||||
|
||||
/// Столбцы после ARRAY JOIN, JOIN и/или агрегации.
|
||||
/// Columns after ARRAY JOIN, JOIN, and/or aggregation.
|
||||
NamesAndTypesList aggregated_columns;
|
||||
|
||||
/// Таблица, из которой делается запрос.
|
||||
/// The table from which the query is made.
|
||||
const StoragePtr storage;
|
||||
|
||||
bool has_aggregation = false;
|
||||
@ -167,14 +167,14 @@ private:
|
||||
|
||||
SubqueriesForSets subqueries_for_sets;
|
||||
|
||||
/// NOTE: Пока поддерживается только один JOIN на запрос.
|
||||
/// NOTE: So far, only one JOIN per query is supported.
|
||||
|
||||
/** Запрос вида SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k
|
||||
* Соединение делается по столбцу k.
|
||||
* Во время JOIN-а,
|
||||
* - в "правой" таблице, он будет доступен по алиасу k, так как было выполнено действие Project для подзапроса.
|
||||
* - в "левой" таблице, он будет доступен по имени expr(x), так как ещё не было выполнено действие Project.
|
||||
* Надо запомнить оба этих варианта.
|
||||
/** Query of the form `SELECT expr(x) AS FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
|
||||
* The join is made by column k.
|
||||
* During the JOIN,
|
||||
* - in the "right" table, it will be available by alias `k`, since `Project` action for the subquery was executed.
|
||||
* - in the "left" table, it will be accessible by the name `expr(x)`, since `Project` action has not been executed yet.
|
||||
* You must remember both of these options.
|
||||
*/
|
||||
Names join_key_names_left;
|
||||
Names join_key_names_right;
|
||||
@ -187,21 +187,21 @@ private:
|
||||
using SetOfASTs = std::set<const IAST *>;
|
||||
using MapOfASTs = std::map<ASTPtr, ASTPtr>;
|
||||
|
||||
/// Какой столбец нужно по-ARRAY-JOIN-ить, чтобы получить указанный.
|
||||
/// Например, для SELECT s.v ... ARRAY JOIN a AS s сюда попадет "s.v" -> "a.v".
|
||||
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.
|
||||
/// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v".
|
||||
NameToNameMap array_join_result_to_source;
|
||||
|
||||
/// Для секции ARRAY JOIN отображение из алиаса в полное имя столбца.
|
||||
/// Например, для ARRAY JOIN [1,2] AS b сюда попадет "b" -> "array(1,2)".
|
||||
/// For the ARRAY JOIN section, mapping from the alias to the full column name.
|
||||
/// For example, for `ARRAY JOIN [1,2] AS b` "b" -> "array(1,2)" will enter here.
|
||||
NameToNameMap array_join_alias_to_name;
|
||||
|
||||
/// Обратное отображение для array_join_alias_to_name.
|
||||
/// The backward mapping for array_join_alias_to_name.
|
||||
NameToNameMap array_join_name_to_alias;
|
||||
|
||||
/// Нужно ли подготавливать к выполнению глобальные подзапросы при анализировании запроса.
|
||||
/// Do I need to prepare for execution global subqueries when analyzing the query.
|
||||
bool do_global;
|
||||
|
||||
/// Все новые временные таблицы, полученные при выполнении подзапросов GLOBAL IN/JOIN.
|
||||
/// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries.
|
||||
Tables external_tables;
|
||||
size_t external_table_id = 1;
|
||||
|
||||
@ -211,22 +211,22 @@ private:
|
||||
static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols);
|
||||
NamesAndTypesList::iterator findColumn(const String & name) { return findColumn(name, columns); }
|
||||
|
||||
/** Из списка всех доступных столбцов таблицы (columns) удалить все ненужные.
|
||||
* Заодно, сформировать множество неизвестных столбцов (unknown_required_columns),
|
||||
* а также столбцов, добавленных JOIN-ом (columns_added_by_join).
|
||||
/** Remove all unnecessary columns from the list of all available columns of the table (`columns`).
|
||||
* At the same time, form a set of unknown columns (`unknown_required_columns`),
|
||||
* as well as the columns added by JOIN (`columns_added_by_join`).
|
||||
*/
|
||||
void collectUsedColumns();
|
||||
|
||||
/** Найти столбцы, получаемые путём JOIN-а.
|
||||
/** Find the columns that are obtained by JOIN.
|
||||
*/
|
||||
void collectJoinedColumns(NameSet & joined_columns, NamesAndTypesList & joined_columns_name_type);
|
||||
|
||||
/** Создать словарь алиасов.
|
||||
/** Create a dictionary of aliases.
|
||||
*/
|
||||
void addASTAliases(ASTPtr & ast, int ignore_levels = 0);
|
||||
|
||||
/** Для узлов-звёздочек - раскрыть их в список всех столбцов.
|
||||
* Для узлов-литералов - подставить алиасы.
|
||||
/** For star nodes(`*`), expand them to a list of all columns.
|
||||
* For literal nodes, substitute aliases.
|
||||
*/
|
||||
void normalizeTree();
|
||||
void normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, size_t level);
|
||||
@ -234,7 +234,7 @@ private:
|
||||
/// Eliminates injective function calls and constant expressions from group by statement
|
||||
void optimizeGroupBy();
|
||||
|
||||
/// Удалить из ORDER BY повторяющиеся элементы.
|
||||
/// Remove duplicate items from ORDER BY.
|
||||
void optimizeOrderBy();
|
||||
|
||||
void optimizeLimitBy();
|
||||
@ -244,25 +244,25 @@ private:
|
||||
void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast, Aliases & aliases) const;
|
||||
bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const;
|
||||
|
||||
/// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn.
|
||||
/// Transform the value enumeration or subquery into ASTSet. `node` - `in` or `notIn` function.
|
||||
void makeSet(ASTFunction * node, const Block & sample_block);
|
||||
|
||||
/// Добавляет список ALIAS столбцов из таблицы
|
||||
/// Adds a list of ALIAS columns from the table
|
||||
void addAliasColumns();
|
||||
|
||||
/// Замена скалярных подзапросов на значения-константы.
|
||||
/// Replacing scalar subqueries with constant values.
|
||||
void executeScalarSubqueries();
|
||||
void executeScalarSubqueriesImpl(ASTPtr & ast);
|
||||
|
||||
/// Находит глобальные подзапросы в секциях GLOBAL IN/JOIN. Заполняет external_tables.
|
||||
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
|
||||
void initGlobalSubqueriesAndExternalTables();
|
||||
void initGlobalSubqueries(ASTPtr & ast);
|
||||
|
||||
/// Находит в запросе использования внешних таблиц (в виде идентификаторов таблиц). Заполняет external_tables.
|
||||
/// Finds in the query the usage of external tables (as table identifiers). Fills in external_tables.
|
||||
void findExternalTables(ASTPtr & ast);
|
||||
|
||||
/** Инициализировать InterpreterSelectQuery для подзапроса в секции GLOBAL IN/JOIN,
|
||||
* создать временную таблицу типа Memory и запомнить это в словаре external_tables.
|
||||
/** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section,
|
||||
* create a temporary table of type Memory and store it in the external_tables dictionary.
|
||||
*/
|
||||
void addExternalStorage(ASTPtr & subquery_or_table_name);
|
||||
|
||||
@ -279,35 +279,35 @@ private:
|
||||
|
||||
void getActionsBeforeAggregation(ASTPtr ast, ExpressionActionsPtr & actions, bool no_subqueries);
|
||||
|
||||
/** Добавить ключи агрегации в aggregation_keys, агрегатные функции в aggregate_descriptions,
|
||||
* Создать набор столбцов aggregated_columns, получаемых после агрегации, если она есть,
|
||||
* или после всех действий, которые обычно выполняются до агрегации.
|
||||
* Установить has_aggregation = true, если есть GROUP BY или хотя бы одна агрегатная функция.
|
||||
/** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions,
|
||||
* Create a set of columns aggregated_columns resulting after the aggregation, if any,
|
||||
* or after all the actions that are normally performed before aggregation.
|
||||
* Set has_aggregation = true if there is GROUP BY or at least one aggregate function.
|
||||
*/
|
||||
void analyzeAggregation();
|
||||
void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions);
|
||||
void assertNoAggregates(const ASTPtr & ast, const char * description);
|
||||
|
||||
/** Получить множество нужных столбцов для чтения из таблицы.
|
||||
* При этом, столбцы, указанные в ignored_names, считаются ненужными. И параметр ignored_names может модифицироваться.
|
||||
* Множество столбцов available_joined_columns - столбцы, доступные из JOIN-а, они не нужны для чтения из основной таблицы.
|
||||
* Положить в required_joined_columns множество столбцов, доступных из JOIN-а и востребованных.
|
||||
/** Get a set of necessary columns to read from the table.
|
||||
* In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified.
|
||||
* The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table.
|
||||
* Put in required_joined_columns the set of columns available from JOIN and needed.
|
||||
*/
|
||||
void getRequiredColumnsImpl(ASTPtr ast,
|
||||
NameSet & required_columns, NameSet & ignored_names,
|
||||
const NameSet & available_joined_columns, NameSet & required_joined_columns);
|
||||
|
||||
/// Получить таблицу, из которой идет запрос
|
||||
/// Get the table from which the query is made
|
||||
StoragePtr getTable();
|
||||
|
||||
/// columns - столбцы, присутствующие до начала преобразований.
|
||||
/// columns - the columns that are present before the transformations begin.
|
||||
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
|
||||
|
||||
void assertSelect() const;
|
||||
void assertAggregation() const;
|
||||
|
||||
/** Создать Set из явного перечисления значений в запросе.
|
||||
* Если create_ordered_set = true - создать структуру данных, подходящую для использования индекса.
|
||||
/** Create Set from an explicit enumeration of values in the query.
|
||||
* If create_ordered_set = true - create a data structure suitable for using the index.
|
||||
*/
|
||||
void makeExplicitSet(ASTFunction * node, const Block & sample_block, bool create_ordered_set);
|
||||
void makeSetsForIndexImpl(ASTPtr & node, const Block & sample_block);
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Dictionaries/IDictionarySource.h>
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <ext/scope_guard.hpp>
|
||||
#include <ext/scope_guard.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Poco/Glob.h>
|
||||
#include <Poco/File.h>
|
||||
|
@ -6,14 +6,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Интерфейс интерпретаторов разных запросов.
|
||||
/** Interpreters interface for different queries.
|
||||
*/
|
||||
class IInterpreter
|
||||
{
|
||||
public:
|
||||
/** Для запросов, возвращающих результат (SELECT и похожие), устанавливает в BlockIO поток, из которого можно будет читать этот результат.
|
||||
* Для запросов, принимающих данные (INSERT), устанавливает в BlockIO поток, куда можно писать данные.
|
||||
* Для запросов, которые не требуют данные и ничего не возвращают, BlockIO будет пустым.
|
||||
/** For queries that return a result (SELECT and similar), sets in BlockIO a stream from which you can read this result.
|
||||
* For queries that receive data (INSERT), sets a thread in BlockIO where you can write data.
|
||||
* For queries that do not require data and return nothing, BlockIO will be empty.
|
||||
*/
|
||||
virtual BlockIO execute() = 0;
|
||||
|
||||
|
@ -10,8 +10,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Позволяет добавить или удалить столбец в таблице.
|
||||
* Также позволяет осуществить манипуляции с партициями таблиц семейства MergeTree.
|
||||
/** Allows you add or remove a column in the table.
|
||||
* It also allows you to manipulate the partitions of the MergeTree family tables.
|
||||
*/
|
||||
class InterpreterAlterQuery : public IInterpreter
|
||||
{
|
||||
@ -37,13 +37,13 @@ private:
|
||||
|
||||
Field partition;
|
||||
Field column_name;
|
||||
bool detach = false; /// true для DETACH PARTITION.
|
||||
bool detach = false; /// true for DETACH PARTITION.
|
||||
|
||||
bool part = false;
|
||||
|
||||
String from; /// Для FETCH PARTITION - путь в ZK к шарду, с которого скачивать партицию.
|
||||
String from; /// For FETCH PARTITION - path in ZK to the shard, from which to download the partition.
|
||||
|
||||
/// Для RESHARD PARTITION.
|
||||
/// For RESHARD PARTITION.
|
||||
Field last_partition;
|
||||
WeightedZooKeeperPaths weighted_zookeeper_paths;
|
||||
ASTPtr sharding_key_expr;
|
||||
|
@ -534,6 +534,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
context.getDatabase(database_name)->createTable(table_name, res, query_ptr, storage_name, context.getSettingsRef());
|
||||
}
|
||||
|
||||
res->startup();
|
||||
|
||||
/// If the CREATE SELECT query is, insert the data into the table
|
||||
if (create.select && storage_name != "View" && (storage_name != "MaterializedView" || create.is_populate))
|
||||
{
|
||||
|
@ -10,17 +10,17 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Интерпретирует запрос INSERT.
|
||||
/** Interprets the INSERT query.
|
||||
*/
|
||||
class InterpreterInsertQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_);
|
||||
|
||||
/** Подготовить запрос к выполнению. Вернуть потоки блоков
|
||||
* - поток, в который можно писать данные для выполнения запроса, если INSERT;
|
||||
* - поток, из которого можно читать результат выполнения запроса, если SELECT и подобные;
|
||||
* Или ничего, если запрос INSERT SELECT (самодостаточный запрос - не принимает входные данные, не отдаёт результат).
|
||||
/** Prepare a request for execution. Return block streams
|
||||
* - the stream into which you can write data to execute the query, if INSERT;
|
||||
* - the stream from which you can read the result of the query, if SELECT and similar;
|
||||
* Or nothing if the request INSERT SELECT (self-sufficient query - does not accept the input data, does not return the result).
|
||||
*/
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -17,27 +17,27 @@ class ASTSelectQuery;
|
||||
struct SubqueryForSet;
|
||||
|
||||
|
||||
/** Интерпретирует запрос SELECT. Возвращает поток блоков с результатами выполнения запроса до стадии to_stage.
|
||||
/** Interprets the SELECT query. Returns the stream of blocks with the results of the query before `to_stage` stage.
|
||||
*/
|
||||
class InterpreterSelectQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
/** to_stage
|
||||
* - стадия, до которой нужно выполнить запрос. По-умолчанию - до конца.
|
||||
* Можно выполнить до промежуточного состояния агрегации, которые объединяются с разных серверов при распределённой обработке запроса.
|
||||
/** `to_stage`
|
||||
* - the stage to which the query is to be executed. By default - till to the end.
|
||||
* You can perform till the intermediate aggregation state, which are combined from different servers for distributed query processing.
|
||||
*
|
||||
* subquery_depth
|
||||
* - для контроля ограничений на глубину вложенности подзапросов. Для подзапросов передаётся значение, увеличенное на единицу.
|
||||
* - to control the restrictions on the depth of nesting of subqueries. For subqueries, a value that is incremented by one is passed.
|
||||
*
|
||||
* input
|
||||
* - если задан - читать не из таблицы, указанной в запросе, а из готового источника.
|
||||
* - if given - read not from the table specified in the query, but from ready source.
|
||||
*
|
||||
* required_column_names
|
||||
* - удалить из запроса все столбцы кроме указанных - используется для удаления ненужных столбцов из подзапросов.
|
||||
* - delete all columns except the specified ones from the query - it is used to delete unnecessary columns from subqueries.
|
||||
*
|
||||
* table_column_names
|
||||
* - список доступных столбцов таблицы.
|
||||
* Используется, например, совместно с указанием input.
|
||||
* - the list of available columns of the table.
|
||||
* Used, for example, with reference to `input`.
|
||||
*/
|
||||
|
||||
InterpreterSelectQuery(
|
||||
@ -66,12 +66,12 @@ public:
|
||||
|
||||
~InterpreterSelectQuery();
|
||||
|
||||
/** Выполнить запрос, возможно являющиийся цепочкой UNION ALL.
|
||||
* Получить поток блоков для чтения
|
||||
/** Execute a query, possibly part of UNION ALL chain.
|
||||
* Get the stream of blocks to read
|
||||
*/
|
||||
BlockIO execute() override;
|
||||
|
||||
/** Выполнить запрос без объединения потоков, если это возможно.
|
||||
/** Execute the query without union of threads, if it is possible.
|
||||
*/
|
||||
const BlockInputStreams & executeWithoutUnion();
|
||||
|
||||
@ -84,8 +84,8 @@ public:
|
||||
|
||||
private:
|
||||
/**
|
||||
* - Оптимизация, если объект создаётся только, чтобы вызвать getSampleBlock(): учитываем только первый SELECT цепочки UNION ALL, потому что
|
||||
* первый SELECT достаточен для определения нужных столбцов.
|
||||
* - Optimization if an object is created only to call getSampleBlock(): consider only the first SELECT of the UNION ALL chain, because
|
||||
* the first SELECT is sufficient to determine the required columns.
|
||||
*/
|
||||
struct OnlyAnalyzeTag {};
|
||||
InterpreterSelectQuery(
|
||||
@ -97,12 +97,12 @@ private:
|
||||
void basicInit(BlockInputStreamPtr input);
|
||||
void initQueryAnalyzer();
|
||||
|
||||
/// Выполнить один запрос SELECT из цепочки UNION ALL.
|
||||
/// Execute one SELECT query from the UNION ALL chain.
|
||||
void executeSingleQuery();
|
||||
|
||||
/** Оставить в каждом запросе цепочки UNION ALL только нужные столбцы секции SELECT.
|
||||
* Однако, если используется хоть один DISTINCT в цепочке, то все столбцы считаются нужными,
|
||||
* так как иначе DISTINCT работал бы по-другому.
|
||||
/** Leave only the necessary columns of the SELECT section in each query of the UNION ALL chain.
|
||||
* However, if you use at least one DISTINCT in the chain, then all the columns are considered necessary,
|
||||
* since otherwise DISTINCT would work differently.
|
||||
*
|
||||
* Always leave arrayJoin, because it changes number of rows.
|
||||
*
|
||||
@ -112,23 +112,23 @@ private:
|
||||
*/
|
||||
void rewriteExpressionList(const Names & required_column_names);
|
||||
|
||||
/// Содержит ли запрос хотя бы один астериск?
|
||||
/// Does the request contain at least one asterisk?
|
||||
bool hasAsterisk() const;
|
||||
|
||||
// Переименовать столбцы каждого запроса цепочки UNION ALL в такие же имена, как в первом запросе.
|
||||
// Rename the columns of each query for the UNION ALL chain into the same names as in the first query.
|
||||
void renameColumns();
|
||||
|
||||
/** Из какой таблицы читать. При JOIN, возвращается "левая" таблица.
|
||||
/** From which table to read. With JOIN, the "left" table is returned.
|
||||
*/
|
||||
void getDatabaseAndTableNames(String & database_name, String & table_name);
|
||||
|
||||
/** Выбрать из списка столбцов какой-нибудь, лучше - минимального размера.
|
||||
/** Select from the list of columns any, better - with minimum size.
|
||||
*/
|
||||
String getAnyColumn();
|
||||
|
||||
/// Разные стадии выполнения запроса.
|
||||
/// Different stages of query execution.
|
||||
|
||||
/// Вынимает данные из таблицы. Возвращает стадию, до которой запрос был обработан в Storage.
|
||||
/// Fetch data from the table. Returns the stage to which the query was processed in Storage.
|
||||
QueryProcessingStage::Enum executeFetchColumns();
|
||||
|
||||
void executeWhere(ExpressionActionsPtr expression);
|
||||
@ -156,11 +156,11 @@ private:
|
||||
|
||||
void ignoreWithTotals();
|
||||
|
||||
/** Если в запросе SELECT есть секция SETTINGS, то применить настройки из неё.
|
||||
/** If there is a SETTINGS section in the SELECT query, then apply settings from it.
|
||||
*
|
||||
* Секция SETTINGS - настройки для конкретного запроса.
|
||||
* Обычно настройки могут быть переданы другими способами, не внутри запроса.
|
||||
* Но использование такой секции оправдано, если нужно задать настройки для одного подзапроса.
|
||||
* Section SETTINGS - settings for a specific query.
|
||||
* Normally, the settings can be passed in other ways, not inside the query.
|
||||
* But the use of this section is justified if you need to set the settings for one subquery.
|
||||
*/
|
||||
void initSettings();
|
||||
|
||||
@ -172,33 +172,33 @@ private:
|
||||
std::unique_ptr<ExpressionAnalyzer> query_analyzer;
|
||||
NamesAndTypesList table_column_names;
|
||||
|
||||
/** Потоки данных.
|
||||
* Исходные потоки данных получаются в функции executeFetchColumns.
|
||||
* Затем они преобразуются (оборачиваются в другие потоки) с помощью функций execute*,
|
||||
* чтобы получить целый конвейер выполнения запроса.
|
||||
/** Streams of data.
|
||||
* The source data streams are produced in the executeFetchColumns function.
|
||||
* Then they are converted (wrapped in other streams) using the `execute*` functions,
|
||||
* to get the whole pipeline running the query.
|
||||
*/
|
||||
BlockInputStreams streams;
|
||||
|
||||
/** При выполнении FULL или RIGHT JOIN, здесь будет поток данных, из которого можно прочитать "неприсоединённые" строки.
|
||||
* Он имеет особое значение, так как чтение из него должно осуществляться после чтения из основных потоков.
|
||||
* Он подклеивается к основным потокам в UnionBlockInputStream или ParallelAggregatingBlockInputStream.
|
||||
/** When executing FULL or RIGHT JOIN, there will be a data stream from which you can read "not joined" rows.
|
||||
* It has a special meaning, since reading from it should be done after reading from the main streams.
|
||||
* It is joined to the main streams in UnionBlockInputStream or ParallelAggregatingBlockInputStream.
|
||||
*/
|
||||
BlockInputStreamPtr stream_with_non_joined_data;
|
||||
|
||||
/// Являемся ли мы первым запросом SELECT цепочки UNION ALL?
|
||||
/// Is it the first SELECT query of the UNION ALL chain?
|
||||
bool is_first_select_inside_union_all;
|
||||
|
||||
/// Объект создан только для анализа запроса.
|
||||
/// The object was created only for query analysis.
|
||||
bool only_analyze = false;
|
||||
|
||||
/// Следующий запрос SELECT в цепочке UNION ALL, если есть.
|
||||
/// The next SELECT query in the UNION ALL chain, if any.
|
||||
std::unique_ptr<InterpreterSelectQuery> next_select_in_union_all;
|
||||
|
||||
/// Таблица, откуда читать данные, если не подзапрос.
|
||||
/// Table from where to read data, if not subquery.
|
||||
StoragePtr storage;
|
||||
TableStructureReadLockPtr table_lock;
|
||||
|
||||
/// Выполнить объединение потоков внутри запроса SELECT?
|
||||
/// Do union of streams within a SELECT query?
|
||||
bool union_within_single_query = false;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
@ -22,7 +22,7 @@ namespace ErrorCodes
|
||||
extern const int NO_SUCH_INTERSERVER_IO_ENDPOINT;
|
||||
}
|
||||
|
||||
/** Местонахождение сервиса.
|
||||
/** Location of the service.
|
||||
*/
|
||||
struct InterserverIOEndpointLocation
|
||||
{
|
||||
@ -32,7 +32,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
/// Создаёт местонахождение на основе его сериализованного представления.
|
||||
/// Creates a location based on its serialized representation.
|
||||
InterserverIOEndpointLocation(const std::string & serialized_location)
|
||||
{
|
||||
ReadBufferFromString buf(serialized_location);
|
||||
@ -42,7 +42,7 @@ public:
|
||||
assertEOF(buf);
|
||||
}
|
||||
|
||||
/// Сериализует местонахождение.
|
||||
/// Serializes the location.
|
||||
std::string toString() const
|
||||
{
|
||||
std::string serialized_location;
|
||||
@ -60,7 +60,7 @@ public:
|
||||
UInt16 port;
|
||||
};
|
||||
|
||||
/** Обработчик запросов от других серверов.
|
||||
/** Query processor from other servers.
|
||||
*/
|
||||
class InterserverIOEndpoint
|
||||
{
|
||||
@ -72,15 +72,15 @@ public:
|
||||
void cancel() { is_cancelled = true; }
|
||||
|
||||
protected:
|
||||
/// Нужно остановить передачу данных.
|
||||
/// You need to stop the data transfer.
|
||||
std::atomic<bool> is_cancelled {false};
|
||||
};
|
||||
|
||||
using InterserverIOEndpointPtr = std::shared_ptr<InterserverIOEndpoint>;
|
||||
|
||||
|
||||
/** Сюда можно зарегистрировать сервис, обрататывающий запросы от других серверов.
|
||||
* Используется для передачи кусков в ReplicatedMergeTree.
|
||||
/** Here you can register a service that processes requests from other servers.
|
||||
* Used to transfer chunks in ReplicatedMergeTree.
|
||||
*/
|
||||
class InterserverIOHandler
|
||||
{
|
||||
@ -116,7 +116,7 @@ private:
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
/// В конструкторе вызывает addEndpoint, в деструкторе - removeEndpoint.
|
||||
/// In the constructor calls `addEndpoint`, in the destructor - `removeEndpoint`.
|
||||
class InterserverIOEndpointHolder
|
||||
{
|
||||
public:
|
||||
@ -136,8 +136,8 @@ public:
|
||||
try
|
||||
{
|
||||
handler.removeEndpoint(name);
|
||||
/// После уничтожения объекта, endpoint ещё может жить, так как владение им захватывается на время обработки запроса,
|
||||
/// см. InterserverIOHTTPHandler.cpp
|
||||
/// After destroying the object, `endpoint` can still live, since its ownership is acquired during the processing of the request,
|
||||
/// see InterserverIOHTTPHandler.cpp
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -16,20 +16,20 @@ struct Settings;
|
||||
class ASTFunction;
|
||||
class ASTSelectQuery;
|
||||
|
||||
/** Этот класс предоставляет функции для оптимизации логических выражений внутри запросов.
|
||||
/** This class provides functions for optimizing boolean expressions within queries.
|
||||
*
|
||||
* Для простоты назовём однородной OR-цепочой любое выражение имеющее следующую структуру:
|
||||
* For simplicity, we call a homogeneous OR-chain any expression having the following structure:
|
||||
* expr = x1 OR ... OR expr = xN
|
||||
* где expr - произвольное выражение и x1, ..., xN - литералы одного типа
|
||||
* where `expr` is an arbitrary expression and x1, ..., xN are literals of the same type
|
||||
*/
|
||||
class LogicalExpressionsOptimizer final
|
||||
{
|
||||
public:
|
||||
/// Конструктор. Принимает корень DAG запроса.
|
||||
/// Constructor. Accepts the root of the query DAG.
|
||||
LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, const Settings & settings_);
|
||||
|
||||
/** Заменить все довольно длинные однородные OR-цепочки expr = x1 OR ... OR expr = xN
|
||||
* на выражения expr IN (x1, ..., xN).
|
||||
/** Replace all rather long homogeneous OR-chains expr = x1 OR ... OR expr = xN
|
||||
* on the expressions `expr` IN (x1, ..., xN).
|
||||
*/
|
||||
void perform();
|
||||
|
||||
@ -37,7 +37,7 @@ public:
|
||||
LogicalExpressionsOptimizer & operator=(const LogicalExpressionsOptimizer &) = delete;
|
||||
|
||||
private:
|
||||
/** Функция OR с выражением.
|
||||
/** The OR function with the expression.
|
||||
*/
|
||||
struct OrWithExpression
|
||||
{
|
||||
@ -60,27 +60,27 @@ private:
|
||||
using DisjunctiveEqualityChain = DisjunctiveEqualityChainsMap::value_type;
|
||||
|
||||
private:
|
||||
/** Собрать информация про все равенства входящие в цепочки OR (не обязательно однородные).
|
||||
* Эта информация сгруппирована по выражению, которое стоит в левой части равенства.
|
||||
/** Collect information about all the equations in the OR chains (not necessarily homogeneous).
|
||||
* This information is grouped by the expression that is on the left side of the equation.
|
||||
*/
|
||||
void collectDisjunctiveEqualityChains();
|
||||
|
||||
/** Проверить, что множество равенств expr = x1, ..., expr = xN выполняет два следующих требования:
|
||||
* 1. Оно не слишком маленькое
|
||||
* 2. x1, ... xN имеют один и тот же тип
|
||||
/** Check that the set of equalities expr = x1, ..., expr = xN fulfills the following two requirements:
|
||||
* 1. It's not too small
|
||||
* 2. x1, ... xN have the same type
|
||||
*/
|
||||
bool mayOptimizeDisjunctiveEqualityChain(const DisjunctiveEqualityChain & chain) const;
|
||||
|
||||
/// Вставить выражение IN в OR-цепочку.
|
||||
/// Insert the IN expression into the OR chain.
|
||||
void addInExpression(const DisjunctiveEqualityChain & chain);
|
||||
|
||||
/// Удалить равенства, которые были заменены выражениями IN.
|
||||
/// Delete the equalities that were replaced by the IN expressions.
|
||||
void cleanupOrExpressions();
|
||||
|
||||
/// Удалить выражения OR, которые имеют только один операнд.
|
||||
/// Delete OR expressions that have only one operand.
|
||||
void fixBrokenOrExpressions();
|
||||
|
||||
/// Восстановить исходный порядок столбцов после оптимизации.
|
||||
/// Restore the original column order after optimization.
|
||||
void reorderColumns();
|
||||
|
||||
private:
|
||||
@ -91,13 +91,13 @@ private:
|
||||
private:
|
||||
ASTSelectQuery * select_query;
|
||||
const Settings & settings;
|
||||
/// Информация про OR-цепочки внутри запроса.
|
||||
/// Information about the OR-chains inside the query.
|
||||
DisjunctiveEqualityChainsMap disjunctive_equality_chains_map;
|
||||
/// Количество обработанных OR-цепочек.
|
||||
/// Number of processed OR-chains.
|
||||
size_t processed_count = 0;
|
||||
/// Родители функций OR.
|
||||
/// Parents of OR functions.
|
||||
FunctionParentMap or_parent_map;
|
||||
/// Позиция каждого столбца.
|
||||
/// The position of each column.
|
||||
ColumnToPosition column_to_position;
|
||||
/// Set of nodes, that was visited.
|
||||
std::unordered_set<void *> visited_nodes;
|
||||
|
@ -60,7 +60,7 @@ private:
|
||||
|
||||
while (true)
|
||||
{
|
||||
/// Если ли хотя бы один более приоритетный запрос?
|
||||
/// Is there at least one more priority query?
|
||||
bool found = false;
|
||||
for (const auto & value : container)
|
||||
{
|
||||
|
@ -77,41 +77,41 @@ private:
|
||||
* We have not implemented such sophisticated behaviour.
|
||||
*/
|
||||
|
||||
/** Типы данных, из которых было создано множество.
|
||||
* При проверке на принадлежность множеству, типы проверяемых столбцов должны с ними совпадать.
|
||||
/** The data types from which the set was created.
|
||||
* When checking for belonging to a set, the types of columns to be checked must match with them.
|
||||
*/
|
||||
DataTypes data_types;
|
||||
|
||||
Logger * log;
|
||||
|
||||
/// Ограничения на максимальный размер множества
|
||||
/// Limitations on the maximum size of the set
|
||||
size_t max_rows;
|
||||
size_t max_bytes;
|
||||
OverflowMode overflow_mode;
|
||||
|
||||
/// Если в левой части IN стоит массив. Проверяем, что хоть один элемент массива лежит в множестве.
|
||||
/// If there is an array on the left side of IN. We check that at least one element of the array presents in the set.
|
||||
void executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const;
|
||||
|
||||
/// Если в левой части набор столбцов тех же типов, что элементы множества.
|
||||
/// If in the left part columns contains the same types as the elements of the set.
|
||||
void executeOrdinary(
|
||||
const ConstColumnPlainPtrs & key_columns,
|
||||
ColumnUInt8::Container_t & vec_res,
|
||||
bool negative,
|
||||
const PaddedPODArray<UInt8> * null_map) const;
|
||||
|
||||
/// Проверить не превышены ли допустимые размеры множества ключей
|
||||
/// Check whether the permissible sizes of keys set reached
|
||||
bool checkSetSizeLimits() const;
|
||||
|
||||
/// Вектор упорядоченных элементов Set.
|
||||
/// Нужен для работы индекса по первичному ключу в операторе IN.
|
||||
/// Vector of ordered elements of `Set`.
|
||||
/// It is necessary for the index to work on the primary key in the IN statement.
|
||||
using OrderedSetElements = std::vector<Field>;
|
||||
using OrderedSetElementsPtr = std::unique_ptr<OrderedSetElements>;
|
||||
OrderedSetElementsPtr ordered_set_elements;
|
||||
|
||||
/** Защищает работу с множеством в функциях insertFromBlock и execute.
|
||||
* Эти функции могут вызываться одновременно из разных потоков только при использовании StorageSet,
|
||||
* и StorageSet вызывает только эти две функции.
|
||||
* Поэтому остальные функции по работе с множеством, не защинены.
|
||||
/** Protects work with the set in the functions `insertFromBlock` and `execute`.
|
||||
* These functions can be called simultaneously from different threads only when using StorageSet,
|
||||
* and StorageSet calls only these two functions.
|
||||
* Therefore, the rest of the functions for working with set are not protected.
|
||||
*/
|
||||
mutable Poco::RWLock rwlock;
|
||||
|
||||
|
@ -18,8 +18,8 @@ namespace DB
|
||||
*/
|
||||
|
||||
|
||||
/// Для случая, когда есть один числовой ключ.
|
||||
template <typename FieldType, typename TData> /// UInt8/16/32/64 для любых типов соответствующей битности.
|
||||
/// For the case where there is one numeric key.
|
||||
template <typename FieldType, typename TData> /// UInt8/16/32/64 for any types with corresponding bit width.
|
||||
struct SetMethodOneNumber
|
||||
{
|
||||
using Data = TData;
|
||||
@ -27,36 +27,36 @@ struct SetMethodOneNumber
|
||||
|
||||
Data data;
|
||||
|
||||
/// Для использования одного Method в разных потоках, используйте разные State.
|
||||
/// To use one `Method` in different threads, use different `State`.
|
||||
struct State
|
||||
{
|
||||
const FieldType * vec;
|
||||
|
||||
/** Вызывается в начале обработки каждого блока.
|
||||
* Устанавливает переменные, необходимые для остальных методов, вызываемых во внутренних циклах.
|
||||
/** Called at the start of each block processing.
|
||||
* Sets the variables required for the other methods called in internal loops.
|
||||
*/
|
||||
void init(const ConstColumnPlainPtrs & key_columns)
|
||||
{
|
||||
vec = &static_cast<const ColumnVector<FieldType> *>(key_columns[0])->getData()[0];
|
||||
}
|
||||
|
||||
/// Достать из ключевых столбцов ключ для вставки в хэш-таблицу.
|
||||
/// Get key from key columns for insertion into hash table.
|
||||
Key getKey(
|
||||
const ConstColumnPlainPtrs & key_columns, /// Ключевые столбцы.
|
||||
size_t keys_size, /// Количество ключевых столбцов.
|
||||
size_t i, /// Из какой строки блока достать ключ.
|
||||
const Sizes & key_sizes) const /// Если ключи фиксированной длины - их длины. Не используется в методах по ключам переменной длины.
|
||||
const ConstColumnPlainPtrs & key_columns, /// Key columns.
|
||||
size_t keys_size, /// Number of key columns.
|
||||
size_t i, /// From what row of the block I get the key.
|
||||
const Sizes & key_sizes) const /// If keys of a fixed length - their lengths. Not used in methods for variable length keys.
|
||||
{
|
||||
return unionCastToUInt64(vec[i]);
|
||||
}
|
||||
};
|
||||
|
||||
/** Разместить дополнительные данные, если это необходимо, в случае, когда в хэш-таблицу был вставлен новый ключ.
|
||||
/** Place additional data, if necessary, in case a new key was inserted into the hash table.
|
||||
*/
|
||||
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
|
||||
};
|
||||
|
||||
/// Для случая, когда есть один строковый ключ.
|
||||
/// For the case where there is one string key.
|
||||
template <typename TData>
|
||||
struct SetMethodString
|
||||
{
|
||||
@ -96,7 +96,7 @@ struct SetMethodString
|
||||
}
|
||||
};
|
||||
|
||||
/// Для случая, когда есть один строковый ключ фиксированной длины.
|
||||
/// For the case when there is one fixed-length string key.
|
||||
template <typename TData>
|
||||
struct SetMethodFixedString
|
||||
{
|
||||
@ -231,7 +231,7 @@ protected:
|
||||
|
||||
}
|
||||
|
||||
/// Для случая, когда все ключи фиксированной длины, и они помещаются в N (например, 128) бит.
|
||||
/// For the case when all keys are of fixed length, and they fit in N (for example, 128) bits.
|
||||
template <typename TData, bool has_nullable_keys_ = false>
|
||||
struct SetMethodKeysFixed
|
||||
{
|
||||
@ -271,7 +271,7 @@ struct SetMethodKeysFixed
|
||||
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, Arena & pool) {}
|
||||
};
|
||||
|
||||
/// Для остальных случаев. По 128 битному хэшу от ключа.
|
||||
/// For other cases. 128 bit hash from the key.
|
||||
template <typename TData>
|
||||
struct SetMethodHashed
|
||||
{
|
||||
@ -300,17 +300,17 @@ struct SetMethodHashed
|
||||
};
|
||||
|
||||
|
||||
/** Разные варианты реализации множества.
|
||||
/** Different implementations of the set.
|
||||
*/
|
||||
struct NonClearableSet
|
||||
{
|
||||
/// TODO Использовать для этих двух вариантов bit- или byte- set.
|
||||
/// TODO Use either bit- or byte-set for these two options.
|
||||
std::unique_ptr<SetMethodOneNumber<UInt8, HashSet<UInt8, TrivialHash, HashTableFixedGrower<8>>>> key8;
|
||||
std::unique_ptr<SetMethodOneNumber<UInt16, HashSet<UInt16, TrivialHash, HashTableFixedGrower<16>>>> key16;
|
||||
|
||||
/** Также для эксперимента проверялась возможность использовать SmallSet,
|
||||
* пока количество элементов в множестве небольшое (и, при необходимости, конвертировать в полноценный HashSet).
|
||||
* Но этот эксперимент показал, что преимущество есть только в редких случаях.
|
||||
/** Also for the experiment was tested the ability to use SmallSet,
|
||||
* as long as the number of elements in the set is small (and, if necessary, converted to a full-fledged HashSet).
|
||||
* But this experiment showed that there is an advantage only in rare cases.
|
||||
*/
|
||||
std::unique_ptr<SetMethodOneNumber<UInt32, HashSet<UInt32, HashCRC32<UInt32>>>> key32;
|
||||
std::unique_ptr<SetMethodOneNumber<UInt64, HashSet<UInt64, HashCRC32<UInt64>>>> key64;
|
||||
@ -323,15 +323,15 @@ struct NonClearableSet
|
||||
/// Support for nullable keys (for DISTINCT implementation).
|
||||
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt128, UInt128HashCRC32>, true>> nullable_keys128;
|
||||
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt256, UInt256HashCRC32>, true>> nullable_keys256;
|
||||
/** В отличие от Aggregator, здесь не используется метод concat.
|
||||
* Это сделано потому что метод hashed, хоть и медленнее, но в данном случае, использует меньше оперативки.
|
||||
* так как при его использовании, сами значения ключей не сохраняются.
|
||||
/** Unlike Aggregator, `concat` method is not used here.
|
||||
* This is done because `hashed` method, although slower, but in this case, uses less RAM.
|
||||
* since when you use it, the key values themselves are not stored.
|
||||
*/
|
||||
};
|
||||
|
||||
struct ClearableSet
|
||||
{
|
||||
/// TODO Использовать для этих двух вариантов bit- или byte- set.
|
||||
/// TODO Use either bit- or byte-set for these two options.
|
||||
std::unique_ptr<SetMethodOneNumber<UInt8, ClearableHashSet<UInt8, TrivialHash, HashTableFixedGrower<8>>>> key8;
|
||||
std::unique_ptr<SetMethodOneNumber<UInt16, ClearableHashSet<UInt16, TrivialHash, HashTableFixedGrower<16>>>> key16;
|
||||
|
||||
@ -346,9 +346,9 @@ struct ClearableSet
|
||||
/// Support for nullable keys (for DISTINCT implementation).
|
||||
std::unique_ptr<SetMethodKeysFixed<ClearableHashSet<UInt128, UInt128HashCRC32>, true>> nullable_keys128;
|
||||
std::unique_ptr<SetMethodKeysFixed<ClearableHashSet<UInt256, UInt256HashCRC32>, true>> nullable_keys256;
|
||||
/** В отличие от Aggregator, здесь не используется метод concat.
|
||||
* Это сделано потому что метод hashed, хоть и медленнее, но в данном случае, использует меньше оперативки.
|
||||
* так как при его использовании, сами значения ключей не сохраняются.
|
||||
/** Unlike Aggregator, `concat` method is not used here.
|
||||
* This is done because `hashed` method, although slower, but in this case, uses less RAM.
|
||||
* since when you use it, the key values themselves are not stored.
|
||||
*/
|
||||
};
|
||||
|
||||
@ -392,7 +392,7 @@ struct SetVariantsTemplate: public Variant
|
||||
void init(Type type_);
|
||||
|
||||
size_t getTotalRowCount() const;
|
||||
/// Считает размер в байтах буфера Set и размер string_pool'а
|
||||
/// Counts the size in bytes of the Set buffer and the size of the `string_pool`
|
||||
size_t getTotalByteCount() const;
|
||||
};
|
||||
|
||||
|
@ -16,7 +16,7 @@ namespace DB
|
||||
*/
|
||||
struct Settings
|
||||
{
|
||||
/// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in С++14.
|
||||
/// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14.
|
||||
/// http://en.cppreference.com/w/cpp/language/aggregate_initialization
|
||||
Settings() {}
|
||||
|
||||
@ -145,7 +145,7 @@ struct Settings
|
||||
M(SettingBool, force_index_by_date, 0) \
|
||||
M(SettingBool, force_primary_key, 0) \
|
||||
\
|
||||
/** In the INSERT query with specified columns, fill in the default values only for columns with explicit DEFAULTs. */ \
|
||||
/** In the INSERT query with specified columns, fill in the default values only for columns with explicit DEFAULTs. */ \
|
||||
M(SettingBool, strict_insert_defaults, 0) \
|
||||
\
|
||||
/** If the maximum size of mark_cache is exceeded, delete only records older than mark_cache_min_lifetime seconds. */ \
|
||||
|
@ -26,11 +26,11 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
/** Одна настройка какого-либо типа.
|
||||
* Хранит внутри себя значение, а также флаг - было ли значение изменено.
|
||||
* Это сделано, чтобы можно было отправлять на удалённые серверы только изменённые (или явно указанные в конфиге) значения.
|
||||
* То есть, если настройка не была указана в конфиге и не была изменена динамически, то она не отправляется на удалённый сервер,
|
||||
* и удалённый сервер будет использовать своё значение по-умолчанию.
|
||||
/** One setting for any type.
|
||||
* Stores a value within itself, as well as a flag - whether the value was changed.
|
||||
* This is done so that you can send to the remote servers only changed settings (or explicitly specified in the config) values.
|
||||
* That is, if the configuration was not specified in the config and was not dynamically changed, it is not sent to the remote server,
|
||||
* and the remote server will use its default value.
|
||||
*/
|
||||
|
||||
template <typename IntType>
|
||||
@ -83,9 +83,9 @@ using SettingInt64 = SettingInt<Int64>;
|
||||
using SettingBool = SettingUInt64;
|
||||
|
||||
|
||||
/** В отличие от SettingUInt64, поддерживает значение 'auto' - количество процессорных ядер без учёта SMT.
|
||||
* Значение 0 так же воспринимается как auto.
|
||||
* При сериализации, auto записывается так же, как 0.
|
||||
/** Unlike SettingUInt64, supports the value of 'auto' - the number of processor cores without taking into account SMT.
|
||||
* A value of 0 is also treated as auto.
|
||||
* When serializing, `auto` is written in the same way as 0.
|
||||
*/
|
||||
struct SettingMaxThreads
|
||||
{
|
||||
@ -100,7 +100,7 @@ struct SettingMaxThreads
|
||||
|
||||
String toString() const
|
||||
{
|
||||
/// Вместо значения auto выводим актуальное значение, чтобы его было легче посмотреть.
|
||||
/// Instead of the `auto` value, we output the actual value to make it easier to see.
|
||||
return DB::toString(value);
|
||||
}
|
||||
|
||||
@ -151,7 +151,7 @@ struct SettingMaxThreads
|
||||
return res;
|
||||
}
|
||||
|
||||
/// Выполняется один раз за всё время. Выполняется из одного потока.
|
||||
/// Executed once for all time. Executed from one thread.
|
||||
UInt64 getAutoValueImpl() const
|
||||
{
|
||||
return getNumberOfPhysicalCPUCores();
|
||||
@ -323,12 +323,12 @@ struct SettingFloat
|
||||
|
||||
enum class LoadBalancing
|
||||
{
|
||||
/// среди реплик с минимальным количеством ошибок выбирается случайная
|
||||
/// among replicas with a minimum number of errors selected randomly
|
||||
RANDOM = 0,
|
||||
/// среди реплик с минимальным количеством ошибок выбирается реплика
|
||||
/// с минимальным количеством отличающихся символов в имени реплики и имени локального хоста
|
||||
/// a replica is selected among the replicas with the minimum number of errors
|
||||
/// with the minimum number of distinguished characters in the replica name and local hostname
|
||||
NEAREST_HOSTNAME,
|
||||
/// реплики перебираются строго по порядку; количество ошибок не имеет значение
|
||||
/// replicas are walked through strictly in order; the number of errors does not matter
|
||||
IN_ORDER,
|
||||
};
|
||||
|
||||
@ -390,16 +390,16 @@ struct SettingLoadBalancing
|
||||
};
|
||||
|
||||
|
||||
/// Какие строки включать в TOTALS.
|
||||
/// Which rows should be included in TOTALS.
|
||||
enum class TotalsMode
|
||||
{
|
||||
BEFORE_HAVING = 0, /// Считать HAVING по всем прочитанным строкам;
|
||||
/// включая не попавшие в max_rows_to_group_by
|
||||
/// и не прошедшие HAVING после группировки.
|
||||
AFTER_HAVING_INCLUSIVE = 1, /// Считать по всем строкам, кроме не прошедших HAVING;
|
||||
/// то есть, включать в TOTALS все строки, не прошедшие max_rows_to_group_by.
|
||||
AFTER_HAVING_EXCLUSIVE = 2, /// Включать только строки, прошедшие и max_rows_to_group_by, и HAVING.
|
||||
AFTER_HAVING_AUTO = 3, /// Автоматически выбирать между INCLUSIVE и EXCLUSIVE,
|
||||
BEFORE_HAVING = 0, /// Count HAVING for all read rows;
|
||||
/// including those not in max_rows_to_group_by
|
||||
/// and have not passed HAVING after grouping.
|
||||
AFTER_HAVING_INCLUSIVE = 1, /// Count on all rows except those that have not passed HAVING;
|
||||
/// that is, to include in TOTALS all the rows that did not pass max_rows_to_group_by.
|
||||
AFTER_HAVING_EXCLUSIVE = 2, /// Include only the rows that passed and max_rows_to_group_by, and HAVING.
|
||||
AFTER_HAVING_AUTO = 3, /// Automatically select between INCLUSIVE and EXCLUSIVE,
|
||||
};
|
||||
|
||||
struct SettingTotalsMode
|
||||
@ -465,13 +465,13 @@ struct SettingTotalsMode
|
||||
}
|
||||
};
|
||||
|
||||
/// Что делать, если ограничение превышено.
|
||||
/// What to do if the limit is exceeded.
|
||||
enum class OverflowMode
|
||||
{
|
||||
THROW = 0, /// Кинуть исключение.
|
||||
BREAK = 1, /// Прервать выполнение запроса, вернуть что есть.
|
||||
ANY = 2, /** Только для GROUP BY: не добавлять новые строки в набор,
|
||||
* но продолжать агрегировать для ключей, успевших попасть в набор.
|
||||
THROW = 0, /// Throw exception.
|
||||
BREAK = 1, /// Abort query execution, return what is.
|
||||
ANY = 2, /** Only for GROUP BY: do not add new rows to the set,
|
||||
* but continue to aggregate for keys that are already in the set.
|
||||
*/
|
||||
};
|
||||
|
||||
@ -613,13 +613,13 @@ struct SettingCompressionMethod
|
||||
}
|
||||
};
|
||||
|
||||
/// Настройка для выполнения распределённых подзапросов внутри секций IN или JOIN.
|
||||
/// The setting for executing distributed subqueries inside IN or JOIN sections.
|
||||
enum class DistributedProductMode
|
||||
{
|
||||
DENY = 0, /// Запретить
|
||||
LOCAL, /// Конвертировать в локальный запрос
|
||||
GLOBAL, /// Конвертировать в глобальный запрос
|
||||
ALLOW /// Разрешить
|
||||
DENY = 0, /// Disable
|
||||
LOCAL, /// Convert to local query
|
||||
GLOBAL, /// Convert to global query
|
||||
ALLOW /// Enable
|
||||
};
|
||||
|
||||
struct SettingDistributedProductMode
|
||||
@ -680,11 +680,11 @@ struct SettingDistributedProductMode
|
||||
}
|
||||
};
|
||||
|
||||
/// Способ выполнения глобальных распределённых подзапросов.
|
||||
/// Method for executing global distributed subqueries.
|
||||
enum class GlobalSubqueriesMethod
|
||||
{
|
||||
PUSH = 0, /// Отправлять данные подзапроса на все удалённые серверы.
|
||||
PULL = 1, /// Удалённые серверы будут скачивать данные подзапроса с сервера-инициатора.
|
||||
PUSH = 0, /// Send the subquery data to all remote servers.
|
||||
PULL = 1, /// Remote servers will download the subquery data from the initiating server.
|
||||
};
|
||||
|
||||
struct SettingGlobalSubqueriesMethod
|
||||
|
@ -22,16 +22,16 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Шаблон цикла агрегации, позволяющий сгенерировать специализированный вариант для конкретной комбинации агрегатных функций.
|
||||
* Отличается от обычного тем, что вызовы агрегатных функций должны инлайниться, а цикл обновления агрегатных функций должен развернуться.
|
||||
/** An aggregation cycle template that allows you to generate a custom variant for a specific combination of aggregate functions.
|
||||
* It differs from the usual one in that calls to aggregate functions should be inlined, and the update cycle of the aggregate functions should be unfold.
|
||||
*
|
||||
* Так как возможных комбинаций слишком много, то не представляется возможным сгенерировать их все заранее.
|
||||
* Этот шаблон предназначен для того, чтобы инстанцировать его в рантайме,
|
||||
* путём запуска компилятора, компиляции shared library и использования её с помощью dlopen.
|
||||
* Since there are too many possible combinations, it is not possible to generate them all in advance.
|
||||
* This template is intended to instantiate it in runtime,
|
||||
* by running the compiler, compiling shared library, and using it with `dlopen`.
|
||||
*/
|
||||
|
||||
|
||||
/** Список типов - для удобного перечисления агрегатных функций.
|
||||
/** List of types - for convenient listing of aggregate functions.
|
||||
*/
|
||||
template <typename... TTail>
|
||||
struct TypeList
|
||||
@ -132,9 +132,9 @@ void AggregateFunctionsCreator::operator()()
|
||||
|
||||
try
|
||||
{
|
||||
/** Может возникнуть исключение при нехватке памяти.
|
||||
* Для того, чтобы потом всё правильно уничтожилось, "откатываем" часть созданных состояний.
|
||||
* Код не очень удобный.
|
||||
/** An exception may occur if there is a shortage of memory.
|
||||
* To ensure that everything is properly destroyed, we "roll back" some of the created states.
|
||||
* The code is not very convenient.
|
||||
*/
|
||||
func->create(aggregate_data + offsets_of_aggregate_states[column_num]);
|
||||
}
|
||||
@ -186,27 +186,27 @@ void NO_INLINE Aggregator::executeSpecializedCase(
|
||||
StringRefs & keys,
|
||||
AggregateDataPtr overflow_row) const
|
||||
{
|
||||
/// Для всех строчек.
|
||||
/// For all rows.
|
||||
typename Method::iterator it;
|
||||
typename Method::Key prev_key;
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
bool inserted; /// Вставили новый ключ, или такой ключ уже был?
|
||||
bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys.
|
||||
bool inserted; /// Inserted a new key, or was this key already?
|
||||
bool overflow = false; /// New key did not fit in the hash table because of no_more_keys.
|
||||
|
||||
/// Получаем ключ для вставки в хэш-таблицу.
|
||||
/// Get the key to insert into the hash table.
|
||||
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
|
||||
|
||||
if (!no_more_keys) /// Вставляем.
|
||||
if (!no_more_keys) /// Insert.
|
||||
{
|
||||
/// Оптимизация для часто повторяющихся ключей.
|
||||
/// Optimization for frequently repeating keys.
|
||||
if (!Method::no_consecutive_keys_optimization)
|
||||
{
|
||||
if (i != 0 && key == prev_key)
|
||||
{
|
||||
AggregateDataPtr value = Method::getAggregateData(it->second);
|
||||
|
||||
/// Добавляем значения в агрегатные функции.
|
||||
/// Add values into aggregate functions.
|
||||
AggregateFunctionsList::forEach(AggregateFunctionsUpdater(
|
||||
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, value, i, aggregates_pool));
|
||||
|
||||
@ -221,21 +221,21 @@ void NO_INLINE Aggregator::executeSpecializedCase(
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Будем добавлять только если ключ уже есть.
|
||||
/// Add only if the key already exists.
|
||||
inserted = false;
|
||||
it = method.data.find(key);
|
||||
if (method.data.end() == it)
|
||||
overflow = true;
|
||||
}
|
||||
|
||||
/// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего.
|
||||
/// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do.
|
||||
if (no_more_keys && overflow && !overflow_row)
|
||||
{
|
||||
method.onExistingKey(key, keys, *aggregates_pool);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
|
||||
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly some stuff related to the key.
|
||||
if (inserted)
|
||||
{
|
||||
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
|
||||
@ -255,7 +255,7 @@ void NO_INLINE Aggregator::executeSpecializedCase(
|
||||
|
||||
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
|
||||
|
||||
/// Добавляем значения в агрегатные функции.
|
||||
/// Add values into the aggregate functions.
|
||||
AggregateFunctionsList::forEach(AggregateFunctionsUpdater(
|
||||
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, value, i, aggregates_pool));
|
||||
}
|
||||
@ -270,7 +270,7 @@ void NO_INLINE Aggregator::executeSpecializedWithoutKey(
|
||||
AggregateColumns & aggregate_columns,
|
||||
Arena * arena) const
|
||||
{
|
||||
/// Оптимизация в случае единственной агрегатной функции count.
|
||||
/// Optimization in the case of a single aggregate function `count`.
|
||||
AggregateFunctionCount * agg_count = params.aggregates_size == 1
|
||||
? typeid_cast<AggregateFunctionCount *>(aggregate_functions[0])
|
||||
: NULL;
|
||||
@ -290,25 +290,25 @@ void NO_INLINE Aggregator::executeSpecializedWithoutKey(
|
||||
}
|
||||
|
||||
|
||||
/** Основной код компилируется с помощью gcc 5.
|
||||
* Но SpecializedAggregator компилируется с помощью clang 3.6 в .so-файл.
|
||||
* Это делается потому что gcc не удаётся заставить инлайнить функции,
|
||||
* которые были девиртуализированы, в конкретном случае, и производительность получается ниже.
|
||||
* А также clang проще распространять для выкладки на серверы.
|
||||
/** The main code is compiled with gcc 5.
|
||||
* But SpecializedAggregator is compiled using clang 3.6 into the .so file.
|
||||
* This is done because gcc can not get functions inlined,
|
||||
* which were de-virtualized, in a particular case, and the performance is lower.
|
||||
* And also it's easier to distribute clang for deploy to the servers.
|
||||
*
|
||||
* После перехода с gcc 4.8 и gnu++1x на gcc 4.9 и gnu++1y (а затем на gcc 5),
|
||||
* при dlopen стала возникать ошибка: undefined symbol: __cxa_pure_virtual
|
||||
* After switching from gcc 4.8 and gnu++1x to gcc 4.9 and gnu++1y (and then to gcc 5),
|
||||
* an error occurred with `dlopen`: undefined symbol: __cxa_pure_virtual
|
||||
*
|
||||
* Скорее всего, это происходит из-за изменившейся версии этого символа:
|
||||
* gcc создаёт в .so символ
|
||||
* Most likely, this is due to the changed version of this symbol:
|
||||
* gcc creates a symbol in .so
|
||||
* U __cxa_pure_virtual@@CXXABI_1.3
|
||||
* а clang создаёт символ
|
||||
* but clang creates a symbol
|
||||
* U __cxa_pure_virtual
|
||||
*
|
||||
* Но нам не принципиально, как будет реализована функция __cxa_pure_virtual,
|
||||
* потому что она не вызывается при нормальной работе программы,
|
||||
* а если вызывается - то программа и так гарантированно глючит.
|
||||
* But it does not matter for us how the __cxa_pure_virtual function will be implemented,
|
||||
* because it is not called during normal program execution,
|
||||
* and if called - then the program is guaranteed buggy.
|
||||
*
|
||||
* Поэтому, мы можем обойти проблему таким образом:
|
||||
* Therefore, we can work around the problem this way
|
||||
*/
|
||||
extern "C" void __attribute__((__visibility__("default"), __noreturn__)) __cxa_pure_virtual() { abort(); };
|
||||
|
11
dbms/src/Interpreters/SystemLog.cpp
Normal file
11
dbms/src/Interpreters/SystemLog.cpp
Normal file
@ -0,0 +1,11 @@
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
SystemLogs::~SystemLogs() = default;
|
||||
|
||||
}
|
@ -18,6 +18,7 @@
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -51,6 +52,21 @@ namespace DB
|
||||
#define DBMS_SYSTEM_LOG_QUEUE_SIZE 1024
|
||||
|
||||
class Context;
|
||||
class QueryLog;
|
||||
class PartLog;
|
||||
|
||||
|
||||
/// System logs should be destroyed in destructor of last Context and before tables,
|
||||
/// because SystemLog destruction makes insert query while flushing data into underlying tables
|
||||
struct SystemLogs
|
||||
{
|
||||
~SystemLogs();
|
||||
|
||||
std::unique_ptr<QueryLog> query_log; /// Used to log queries.
|
||||
std::unique_ptr<PartLog> part_log; /// Used to log operations with parts
|
||||
};
|
||||
|
||||
|
||||
|
||||
|
||||
template <typename LogElement>
|
||||
@ -267,7 +283,7 @@ void SystemLog<LogElement>::prepareTable()
|
||||
|
||||
if (!blocksHaveEqualStructure(actual, expected))
|
||||
{
|
||||
/// Переименовываем существующую таблицу.
|
||||
/// Rename the existing table.
|
||||
int suffix = 0;
|
||||
while (context.isTableExist(database_name, table_name + "_" + toString(suffix)))
|
||||
++suffix;
|
||||
@ -293,7 +309,7 @@ void SystemLog<LogElement>::prepareTable()
|
||||
|
||||
InterpreterRenameQuery(rename, context).execute();
|
||||
|
||||
/// Нужная таблица будет создана.
|
||||
/// The required table will be created.
|
||||
table = nullptr;
|
||||
}
|
||||
else
|
||||
@ -302,7 +318,7 @@ void SystemLog<LogElement>::prepareTable()
|
||||
|
||||
if (!table)
|
||||
{
|
||||
/// Создаём таблицу.
|
||||
/// Create the table.
|
||||
LOG_DEBUG(log, "Creating new table " << description << " for " + LogElement::name());
|
||||
|
||||
auto create = std::make_shared<ASTCreateQuery>();
|
||||
|
@ -21,7 +21,7 @@
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <ext/scope_guard.hpp>
|
||||
#include <ext/scope_guard.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -8,12 +8,12 @@ namespace DB
|
||||
|
||||
class IDataType;
|
||||
|
||||
/** Используется для интерпретации выражений в множестве в IN,
|
||||
* а также в запросе вида INSERT ... VALUES ...
|
||||
/** Used to interpret expressions in a set in IN,
|
||||
* and also in the query of the form INSERT ... VALUES ...
|
||||
*
|
||||
* Чтобы корректно работали выражения вида 1.0 IN (1) или чтобы 1 IN (1, 2.0, 2.5, -1) работало так же, как 1 IN (1, 2).
|
||||
* Проверяет совместимость типов, проверяет попадание значений в диапазон допустимых значений типа, делает преобразование типа.
|
||||
* Если значение не попадает в диапазон - возвращает Null.
|
||||
* To work correctly with expressions of the form `1.0 IN (1)` or, for example, `1 IN (1, 2.0, 2.5, -1)` work the same way as `1 IN (1, 2)`.
|
||||
* Checks for the compatibility of types, checks values fall in the range of valid values of the type, makes type conversion.
|
||||
* If the value does not fall into the range - returns Null.
|
||||
*/
|
||||
Field convertFieldToType(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint = nullptr);
|
||||
|
||||
|
@ -8,13 +8,13 @@ namespace DB
|
||||
|
||||
class IAST;
|
||||
|
||||
/// Получить имя кластера из AST.
|
||||
/** Имя кластера - это имя тега в xml-конфигурации.
|
||||
* Обычно оно парсится как идентификатор. То есть, оно может содержать подчёркивания, но не может содержать дефисы,
|
||||
* при условии, что идентификатор не находится в обратных кавычках.
|
||||
* Но в xml в качестве имени тега более привычно использовать дефисы.
|
||||
* Такое имя будет парситься как выражение с оператором минус - совсем не то, что нужно.
|
||||
* Поэтому, рассмотрим такой случай отдельно.
|
||||
/// Get the cluster name from AST.
|
||||
/** The name of the cluster is the name of the tag in the xml configuration.
|
||||
* Usually it is parsed as an identifier. That is, it can contain underscores, but can not contain hyphens,
|
||||
* provided that the identifier is not in backquotes.
|
||||
* But in xml, as a tag name, it's more common to use hyphens.
|
||||
* This name will be parsed as an expression with an operator minus - not at all what you need.
|
||||
* Therefore, consider this case separately.
|
||||
*/
|
||||
std::string getClusterName(const IAST & node);
|
||||
|
||||
|
@ -7,25 +7,25 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Отсортировать один блок по описанию desc. Если limit != 0, то производится partial sort первых limit строк.
|
||||
/// Sort one block by `description`. If limit != 0, then the partial sort of the first `limit` rows is produced.
|
||||
void sortBlock(Block & block, const SortDescription & description, size_t limit = 0);
|
||||
|
||||
|
||||
/** Используется только в StorageMergeTree для сортировки данных при INSERT-е.
|
||||
* Сортировка стабильная. Это важно для сохранения порядка записей в движке CollapsingMergeTree
|
||||
* - так как на основе порядка записей определяется, удалять ли или оставлять группы строчек при коллапсировании.
|
||||
* Не поддерживаются collations. Не поддерживается частичная сортировка.
|
||||
/** Used only in StorageMergeTree to sort the data with INSERT.
|
||||
* Sorting is stable. This is important for keeping the order of rows in the CollapsingMergeTree engine
|
||||
* - because based on the order of rows it is determined whether to delete or leave groups of rows when collapsing.
|
||||
* Collations are not supported. Partial sorting is not supported.
|
||||
*/
|
||||
void stableSortBlock(Block & block, const SortDescription & description);
|
||||
|
||||
/** То же, что и stableSortBlock, но не сортировать блок, а только рассчитать перестановку значений,
|
||||
* чтобы потом можно было переставить значения столбцов самостоятельно.
|
||||
/** Same as stableSortBlock, but do not sort the block, but only calculate the permutation of the values,
|
||||
* so that you can rearrange the column values yourself.
|
||||
*/
|
||||
void stableGetPermutation(const Block & block, const SortDescription & description, IColumn::Permutation & out_permutation);
|
||||
|
||||
|
||||
/** Быстро проверить, является ли блок уже отсортированным. Если блок не отсортирован - возвращает false максимально быстро.
|
||||
* Не поддерживаются collations.
|
||||
/** Quickly check whether the block is already sorted. If the block is not sorted - returns false as fast as possible.
|
||||
* Collations are not supported.
|
||||
*/
|
||||
bool isAlreadySorted(const Block & block, const SortDescription & description);
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <ext/shared_ptr_helper.hpp>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
@ -27,16 +27,11 @@ namespace DB
|
||||
|
||||
|
||||
/// Simplified version of the StorageDistributed class.
|
||||
class StorageDistributedFake : private ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
|
||||
class StorageDistributedFake : public ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
|
||||
{
|
||||
friend class ext::shared_ptr_helper<StorageDistributedFake>;
|
||||
|
||||
public:
|
||||
static DB::StoragePtr create(const std::string & remote_database_, const std::string & remote_table_, size_t shard_count_)
|
||||
{
|
||||
return make_shared(remote_database_, remote_table_, shard_count_);
|
||||
}
|
||||
|
||||
std::string getName() const override { return "DistributedFake"; }
|
||||
bool isRemote() const override { return true; }
|
||||
size_t getShardCount() const { return shard_count; }
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user