mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
Less dependencies 2 (#378)
* Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2]. * Less dependencies [#CLICKHOUSE-2].
This commit is contained in:
parent
28f4efa461
commit
ed5b521110
@ -1,12 +1,17 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <unordered_map>
|
||||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||||
#include <DB/DataTypes/IDataType.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class IDataType;
|
||||||
|
using DataTypePtr = std::shared_ptr<IDataType>;
|
||||||
|
using DataTypes = std::vector<DataTypePtr>;
|
||||||
|
|
||||||
|
|
||||||
/** Creates an aggregate function by name.
|
/** Creates an aggregate function by name.
|
||||||
*/
|
*/
|
||||||
class AggregateFunctionFactory final
|
class AggregateFunctionFactory final
|
||||||
|
@ -487,55 +487,32 @@ typename NearestFieldType<T>::Type nearestFieldType(const T & x)
|
|||||||
return typename NearestFieldType<T>::Type(x);
|
return typename NearestFieldType<T>::Type(x);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
|
||||||
|
class ReadBuffer;
|
||||||
|
class WriteBuffer;
|
||||||
/// Заглушки, чтобы DBObject-ы с полем типа Array компилировались.
|
|
||||||
#include <mysqlxx/Manip.h>
|
/// Предполагается что у всех элементов массива одинаковый тип.
|
||||||
|
void readBinary(Array & x, ReadBuffer & buf);
|
||||||
namespace mysqlxx
|
|
||||||
{
|
inline void readText(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||||
std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Array & value);
|
inline void readQuoted(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||||
std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Array & value);
|
|
||||||
std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Array & value);
|
/// Предполагается что у всех элементов массива одинаковый тип.
|
||||||
std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Array & value);
|
void writeBinary(const Array & x, WriteBuffer & buf);
|
||||||
|
|
||||||
std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Tuple & value);
|
void writeText(const Array & x, WriteBuffer & buf);
|
||||||
std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Tuple & value);
|
|
||||||
std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Tuple & value);
|
inline void writeQuoted(const Array & x, WriteBuffer & buf) { throw Exception("Cannot write Array quoted.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||||
std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Tuple & value);
|
|
||||||
}
|
void readBinary(Tuple & x, ReadBuffer & buf);
|
||||||
|
|
||||||
|
inline void readText(Tuple & x, ReadBuffer & buf) { throw Exception("Cannot read Tuple.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||||
namespace DB
|
inline void readQuoted(Tuple & x, ReadBuffer & buf) { throw Exception("Cannot read Tuple.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||||
{
|
|
||||||
class ReadBuffer;
|
void writeBinary(const Tuple & x, WriteBuffer & buf);
|
||||||
class WriteBuffer;
|
|
||||||
|
void writeText(const Tuple & x, WriteBuffer & buf);
|
||||||
/// Предполагается что у всех элементов массива одинаковый тип.
|
|
||||||
void readBinary(Array & x, ReadBuffer & buf);
|
inline void writeQuoted(const Tuple & x, WriteBuffer & buf) { throw Exception("Cannot write Tuple quoted.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||||
|
|
||||||
inline void readText(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
|
|
||||||
inline void readQuoted(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
|
|
||||||
|
|
||||||
/// Предполагается что у всех элементов массива одинаковый тип.
|
|
||||||
void writeBinary(const Array & x, WriteBuffer & buf);
|
|
||||||
|
|
||||||
void writeText(const Array & x, WriteBuffer & buf);
|
|
||||||
|
|
||||||
inline void writeQuoted(const Array & x, WriteBuffer & buf) { throw Exception("Cannot write Array quoted.", ErrorCodes::NOT_IMPLEMENTED); }
|
|
||||||
}
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
void readBinary(Tuple & x, ReadBuffer & buf);
|
|
||||||
|
|
||||||
inline void readText(Tuple & x, ReadBuffer & buf) { throw Exception("Cannot read Tuple.", ErrorCodes::NOT_IMPLEMENTED); }
|
|
||||||
inline void readQuoted(Tuple & x, ReadBuffer & buf) { throw Exception("Cannot read Tuple.", ErrorCodes::NOT_IMPLEMENTED); }
|
|
||||||
|
|
||||||
void writeBinary(const Tuple & x, WriteBuffer & buf);
|
|
||||||
|
|
||||||
void writeText(const Tuple & x, WriteBuffer & buf);
|
|
||||||
|
|
||||||
inline void writeQuoted(const Tuple & x, WriteBuffer & buf) { throw Exception("Cannot write Tuple quoted.", ErrorCodes::NOT_IMPLEMENTED); }
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Core/Field.h>
|
#include <DB/Core/Field.h>
|
||||||
|
#include <common/DateLUT.h>
|
||||||
|
|
||||||
|
|
||||||
class SipHash;
|
class SipHash;
|
||||||
|
@ -1,10 +1,12 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
#include <DB/Storages/IStorage.h>
|
|
||||||
#include <DB/Common/PODArray.h>
|
#include <DB/Common/PODArray.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -102,7 +104,7 @@ private:
|
|||||||
size_t pos_global_start = 0;
|
size_t pos_global_start = 0;
|
||||||
size_t block_preferred_size;
|
size_t block_preferred_size;
|
||||||
|
|
||||||
Logger * log = &Logger::get("ColumnGathererStream");
|
Poco::Logger * log;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,10 +1,10 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <vector>
|
||||||
|
#include <memory>
|
||||||
|
#include <functional>
|
||||||
#include <boost/noncopyable.hpp>
|
#include <boost/noncopyable.hpp>
|
||||||
|
|
||||||
#include <DB/Core/Block.h>
|
#include <DB/Core/Block.h>
|
||||||
#include <DB/Core/Progress.h>
|
|
||||||
#include <DB/Storages/IStorage.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -16,6 +16,14 @@ class IBlockInputStream;
|
|||||||
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
||||||
using BlockInputStreams = std::vector<BlockInputStreamPtr>;
|
using BlockInputStreams = std::vector<BlockInputStreamPtr>;
|
||||||
|
|
||||||
|
class TableStructureReadLock;
|
||||||
|
|
||||||
|
using TableStructureReadLockPtr = std::shared_ptr<TableStructureReadLock>;
|
||||||
|
using TableStructureReadLocks = std::vector<TableStructureReadLockPtr>;
|
||||||
|
|
||||||
|
struct Progress;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/** Коллбэк для отслеживания прогресса выполнения запроса.
|
/** Коллбэк для отслеживания прогресса выполнения запроса.
|
||||||
* Используется в IProfilingBlockInputStream и Context-е.
|
* Используется в IProfilingBlockInputStream и Context-е.
|
||||||
@ -84,10 +92,10 @@ public:
|
|||||||
|
|
||||||
/** Не давать изменить таблицу, пока жив поток блоков.
|
/** Не давать изменить таблицу, пока жив поток блоков.
|
||||||
*/
|
*/
|
||||||
void addTableLock(const IStorage::TableStructureReadLockPtr & lock) { table_locks.push_back(lock); }
|
void addTableLock(const TableStructureReadLockPtr & lock) { table_locks.push_back(lock); }
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
IStorage::TableStructureReadLocks table_locks;
|
TableStructureReadLocks table_locks;
|
||||||
|
|
||||||
BlockInputStreams children;
|
BlockInputStreams children;
|
||||||
|
|
||||||
|
@ -1,7 +1,9 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <string>
|
||||||
|
#include <vector>
|
||||||
|
#include <memory>
|
||||||
#include <boost/noncopyable.hpp>
|
#include <boost/noncopyable.hpp>
|
||||||
#include <DB/Storages/IStorage.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -10,6 +12,12 @@ namespace DB
|
|||||||
class Block;
|
class Block;
|
||||||
struct Progress;
|
struct Progress;
|
||||||
|
|
||||||
|
class TableStructureReadLock;
|
||||||
|
using TableStructureReadLockPtr = std::shared_ptr<TableStructureReadLock>;
|
||||||
|
using TableStructureReadLocks = std::vector<TableStructureReadLockPtr>;
|
||||||
|
|
||||||
|
struct Progress;
|
||||||
|
|
||||||
|
|
||||||
/** Interface of stream for writing data (into table, filesystem, network, terminal, etc.)
|
/** Interface of stream for writing data (into table, filesystem, network, terminal, etc.)
|
||||||
*/
|
*/
|
||||||
@ -44,16 +52,16 @@ public:
|
|||||||
|
|
||||||
/** Content-Type to set when sending HTTP response.
|
/** Content-Type to set when sending HTTP response.
|
||||||
*/
|
*/
|
||||||
virtual String getContentType() const { return "text/plain; charset=UTF-8"; }
|
virtual std::string getContentType() const { return "text/plain; charset=UTF-8"; }
|
||||||
|
|
||||||
virtual ~IBlockOutputStream() {}
|
virtual ~IBlockOutputStream() {}
|
||||||
|
|
||||||
/** Don't let to alter table while instance of stream is alive.
|
/** Don't let to alter table while instance of stream is alive.
|
||||||
*/
|
*/
|
||||||
void addTableLock(const IStorage::TableStructureReadLockPtr & lock) { table_locks.push_back(lock); }
|
void addTableLock(const TableStructureReadLockPtr & lock) { table_locks.push_back(lock); }
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
IStorage::TableStructureReadLocks table_locks;
|
TableStructureReadLocks table_locks;
|
||||||
};
|
};
|
||||||
|
|
||||||
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
|
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
|
||||||
|
@ -1,10 +1,8 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Columns/ColumnConst.h>
|
#include <DB/Core/Block.h>
|
||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
#include <DB/DataTypes/DataTypeNullable.h>
|
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
|
||||||
#include <ext/range.hpp>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -19,7 +17,7 @@ public:
|
|||||||
|
|
||||||
void write(const Block & block) override
|
void write(const Block & block) override
|
||||||
{
|
{
|
||||||
output->write(materialize(block));
|
output->write(materialize(block));
|
||||||
}
|
}
|
||||||
|
|
||||||
void flush() override { output->flush(); }
|
void flush() override { output->flush(); }
|
||||||
@ -34,31 +32,7 @@ public:
|
|||||||
private:
|
private:
|
||||||
BlockOutputStreamPtr output;
|
BlockOutputStreamPtr output;
|
||||||
|
|
||||||
static Block materialize(const Block & original_block)
|
static Block materialize(const Block & original_block);
|
||||||
{
|
|
||||||
/// copy block to get rid of const
|
|
||||||
auto block = original_block;
|
|
||||||
|
|
||||||
for (const auto i : ext::range(0, block.columns()))
|
|
||||||
{
|
|
||||||
auto & element = block.safeGetByPosition(i);
|
|
||||||
auto & src = element.column;
|
|
||||||
ColumnPtr converted = src->convertToFullColumnIfConst();
|
|
||||||
if (converted)
|
|
||||||
{
|
|
||||||
src = converted;
|
|
||||||
auto & type = element.type;
|
|
||||||
if (type->isNull())
|
|
||||||
{
|
|
||||||
/// A ColumnNull that is converted to a full column
|
|
||||||
/// has the type DataTypeNullable(DataTypeUInt8).
|
|
||||||
type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return block;
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -7,6 +7,10 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class IBlockOutputStream;
|
||||||
|
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
|
||||||
|
|
||||||
|
|
||||||
/** Пустой поток блоков.
|
/** Пустой поток блоков.
|
||||||
* Но при первой попытке чтения, копирует данные из переданного input-а в переданный output.
|
* Но при первой попытке чтения, копирует данные из переданного input-а в переданный output.
|
||||||
* Это нужно для выполнения запроса INSERT SELECT - запрос копирует данные, но сам ничего не возвращает.
|
* Это нужно для выполнения запроса INSERT SELECT - запрос копирует данные, но сам ничего не возвращает.
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <string>
|
||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
@ -23,7 +24,7 @@ public:
|
|||||||
void write(const Block & block) override;
|
void write(const Block & block) override;
|
||||||
|
|
||||||
void flush() override;
|
void flush() override;
|
||||||
String getContentType() const override { return "application/octet-stream"; }
|
std::string getContentType() const override { return "application/octet-stream"; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
bool is_first = true;
|
bool is_first = true;
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Core/Block.h>
|
||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
|
#include <DB/Core/NamesAndTypes.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -22,14 +23,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void write(const Block & block) override
|
void write(const Block & block) override;
|
||||||
{
|
|
||||||
for (const auto & column : columns)
|
|
||||||
if (block.has(column.name))
|
|
||||||
throw Exception{"Cannot insert column " + column.name, ErrorCodes::ILLEGAL_COLUMN};
|
|
||||||
|
|
||||||
output->write(block);
|
|
||||||
}
|
|
||||||
|
|
||||||
void flush() override { output->flush(); }
|
void flush() override { output->flush(); }
|
||||||
|
|
||||||
|
@ -1,9 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <mutex>
|
||||||
#include <DB/Databases/IDatabase.h>
|
#include <DB/Databases/IDatabase.h>
|
||||||
#include <DB/Storages/IStorage.h>
|
#include <DB/Storages/IStorage.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -19,7 +23,7 @@ protected:
|
|||||||
mutable std::mutex mutex;
|
mutable std::mutex mutex;
|
||||||
Tables tables;
|
Tables tables;
|
||||||
|
|
||||||
Logger * log;
|
Poco::Logger * log;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
|
|
||||||
|
@ -2,6 +2,8 @@
|
|||||||
|
|
||||||
#include <DB/Dictionaries/IDictionary.h>
|
#include <DB/Dictionaries/IDictionary.h>
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include <common/singleton.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -1,7 +1,8 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Dictionaries/Embedded/RegionsHierarchy.h>
|
#include <DB/Dictionaries/Embedded/RegionsHierarchy.h>
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/Exception.h>
|
||||||
|
#include <unordered_map>
|
||||||
|
|
||||||
|
|
||||||
/** Содержит несколько иерархий регионов, загружаемых из нескольких разных файлов.
|
/** Содержит несколько иерархий регионов, загружаемых из нескольких разных файлов.
|
||||||
@ -13,50 +14,20 @@ class RegionsHierarchies
|
|||||||
private:
|
private:
|
||||||
using Container = std::unordered_map<std::string, RegionsHierarchy>;
|
using Container = std::unordered_map<std::string, RegionsHierarchy>;
|
||||||
Container data;
|
Container data;
|
||||||
Logger * log = &Logger::get("RegionsHierarchies");
|
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static constexpr auto required_key = "path_to_regions_hierarchy_file";
|
/** path_to_regions_hierarchy_file in configuration file
|
||||||
|
* должен указывать на файл с иерархией регионов "по-умолчанию". Она будет доступна по пустому ключу.
|
||||||
/** path должен указывать на файл с иерархией регионов "по-умолчанию". Она будет доступна по пустому ключу.
|
|
||||||
* Кроме того, рядом ищутся файлы, к имени которых (до расширения, если есть) добавлен произвольный _suffix.
|
* Кроме того, рядом ищутся файлы, к имени которых (до расширения, если есть) добавлен произвольный _suffix.
|
||||||
* Такие файлы загружаются, и иерархия регионов кладётся по ключу suffix.
|
* Такие файлы загружаются, и иерархия регионов кладётся по ключу suffix.
|
||||||
*
|
*
|
||||||
* Например, если указано /opt/geo/regions_hierarchy.txt,
|
* Например, если указано /opt/geo/regions_hierarchy.txt,
|
||||||
* то будет также загружен файл /opt/geo/regions_hierarchy_ua.txt, если такой есть - он будет доступен по ключу ua.
|
* то будет также загружен файл /opt/geo/regions_hierarchy_ua.txt, если такой есть - он будет доступен по ключу ua.
|
||||||
*/
|
*/
|
||||||
RegionsHierarchies(const std::string & default_path = Poco::Util::Application::instance().config().getString(required_key))
|
RegionsHierarchies();
|
||||||
{
|
|
||||||
LOG_DEBUG(log, "Adding default regions hierarchy from " << default_path);
|
|
||||||
|
|
||||||
data.emplace(std::piecewise_construct,
|
/// Has corresponding section in configuration file.
|
||||||
std::forward_as_tuple(""),
|
static bool isConfigured();
|
||||||
std::forward_as_tuple(default_path));
|
|
||||||
|
|
||||||
std::string basename = Poco::Path(default_path).getBaseName();
|
|
||||||
|
|
||||||
Poco::Path dir_path = Poco::Path(default_path).absolute().parent();
|
|
||||||
|
|
||||||
Poco::DirectoryIterator dir_end;
|
|
||||||
for (Poco::DirectoryIterator dir_it(dir_path); dir_it != dir_end; ++dir_it)
|
|
||||||
{
|
|
||||||
std::string other_basename = dir_it.path().getBaseName();
|
|
||||||
|
|
||||||
if (0 == other_basename.compare(0, basename.size(), basename) && other_basename.size() > basename.size() + 1)
|
|
||||||
{
|
|
||||||
if (other_basename[basename.size()] != '_')
|
|
||||||
continue;
|
|
||||||
|
|
||||||
std::string suffix = other_basename.substr(basename.size() + 1);
|
|
||||||
|
|
||||||
LOG_DEBUG(log, "Adding regions hierarchy from " << dir_it->path() << ", key: " << suffix);
|
|
||||||
|
|
||||||
data.emplace(std::piecewise_construct,
|
|
||||||
std::forward_as_tuple(suffix),
|
|
||||||
std::forward_as_tuple(dir_it->path()));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/** Перезагружает, при необходимости, все иерархии регионов.
|
/** Перезагружает, при необходимости, все иерархии регионов.
|
||||||
|
@ -1,17 +1,10 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Poco/Util/Application.h>
|
#include <vector>
|
||||||
#include <Poco/Exception.h>
|
#include <boost/noncopyable.hpp>
|
||||||
#include <Poco/File.h>
|
#include <common/Common.h>
|
||||||
|
|
||||||
#include <common/logger_useful.h>
|
|
||||||
#include <common/singleton.h>
|
#include <common/singleton.h>
|
||||||
|
|
||||||
#include <DB/IO/ReadBufferFromFile.h>
|
|
||||||
#include <DB/IO/ReadHelpers.h>
|
|
||||||
#include <DB/IO/WriteHelpers.h>
|
|
||||||
|
|
||||||
#include <boost/noncopyable.hpp>
|
|
||||||
|
|
||||||
#define REGION_TYPE_CITY 6
|
#define REGION_TYPE_CITY 6
|
||||||
#define REGION_TYPE_AREA 5
|
#define REGION_TYPE_AREA 5
|
||||||
@ -27,9 +20,7 @@
|
|||||||
class RegionsHierarchy : private boost::noncopyable
|
class RegionsHierarchy : private boost::noncopyable
|
||||||
{
|
{
|
||||||
private:
|
private:
|
||||||
std::string path;
|
time_t file_modification_time = 0;
|
||||||
time_t file_modification_time;
|
|
||||||
Logger * log;
|
|
||||||
|
|
||||||
using RegionID = UInt32;
|
using RegionID = UInt32;
|
||||||
using RegionType = UInt8;
|
using RegionType = UInt8;
|
||||||
@ -66,182 +57,15 @@ private:
|
|||||||
/// регион - глубина в дереве
|
/// регион - глубина в дереве
|
||||||
RegionDepths depths;
|
RegionDepths depths;
|
||||||
|
|
||||||
public:
|
/// path to file with data
|
||||||
RegionsHierarchy(const std::string & path_ = Poco::Util::Application::instance().config().getString("path_to_regions_hierarchy_file"))
|
std::string path;
|
||||||
: path(path_), file_modification_time(0), log(&Logger::get("RegionsHierarchy"))
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
|
public:
|
||||||
|
RegionsHierarchy();
|
||||||
|
RegionsHierarchy(const std::string & path_);
|
||||||
|
|
||||||
/// Перезагружает, при необходимости, иерархию регионов. Непотокобезопасно.
|
/// Перезагружает, при необходимости, иерархию регионов. Непотокобезопасно.
|
||||||
void reload()
|
void reload();
|
||||||
{
|
|
||||||
time_t new_modification_time = Poco::File(path).getLastModified().epochTime();
|
|
||||||
if (new_modification_time <= file_modification_time)
|
|
||||||
return;
|
|
||||||
file_modification_time = new_modification_time;
|
|
||||||
|
|
||||||
LOG_DEBUG(log, "Reloading regions hierarchy");
|
|
||||||
|
|
||||||
const size_t initial_size = 10000;
|
|
||||||
const size_t max_size = 1000000;
|
|
||||||
|
|
||||||
RegionParents new_parents(initial_size);
|
|
||||||
RegionParents new_city(initial_size);
|
|
||||||
RegionParents new_country(initial_size);
|
|
||||||
RegionParents new_area(initial_size);
|
|
||||||
RegionParents new_district(initial_size);
|
|
||||||
RegionParents new_continent(initial_size);
|
|
||||||
RegionParents new_top_continent(initial_size);
|
|
||||||
RegionPopulations new_populations(initial_size);
|
|
||||||
RegionDepths new_depths(initial_size);
|
|
||||||
RegionTypes types(initial_size);
|
|
||||||
|
|
||||||
DB::ReadBufferFromFile in(path);
|
|
||||||
|
|
||||||
RegionID max_region_id = 0;
|
|
||||||
while (!in.eof())
|
|
||||||
{
|
|
||||||
/** Our internal geobase has negative numbers,
|
|
||||||
* that means "this is garbage, ignore this row".
|
|
||||||
*/
|
|
||||||
Int32 read_region_id = 0;
|
|
||||||
Int32 read_parent_id = 0;
|
|
||||||
Int8 read_type = 0;
|
|
||||||
|
|
||||||
DB::readIntText(read_region_id, in);
|
|
||||||
DB::assertChar('\t', in);
|
|
||||||
DB::readIntText(read_parent_id, in);
|
|
||||||
DB::assertChar('\t', in);
|
|
||||||
DB::readIntText(read_type, in);
|
|
||||||
|
|
||||||
/** Далее может быть перевод строки (старый вариант)
|
|
||||||
* или таб, население региона, перевод строки (новый вариант).
|
|
||||||
*/
|
|
||||||
RegionPopulation population = 0;
|
|
||||||
if (!in.eof() && *in.position() == '\t')
|
|
||||||
{
|
|
||||||
++in.position();
|
|
||||||
UInt64 population_big = 0;
|
|
||||||
DB::readIntText(population_big, in);
|
|
||||||
population = population_big > std::numeric_limits<RegionPopulation>::max()
|
|
||||||
? std::numeric_limits<RegionPopulation>::max()
|
|
||||||
: population_big;
|
|
||||||
}
|
|
||||||
DB::assertChar('\n', in);
|
|
||||||
|
|
||||||
if (read_region_id <= 0 || read_type < 0)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
RegionID region_id = read_region_id;
|
|
||||||
RegionID parent_id = 0;
|
|
||||||
|
|
||||||
if (read_parent_id >= 0)
|
|
||||||
parent_id = read_parent_id;
|
|
||||||
|
|
||||||
RegionType type = read_type;
|
|
||||||
|
|
||||||
if (region_id > max_region_id)
|
|
||||||
{
|
|
||||||
if (region_id > max_size)
|
|
||||||
throw DB::Exception("Region id is too large: " + DB::toString(region_id) + ", should be not more than " + DB::toString(max_size));
|
|
||||||
|
|
||||||
max_region_id = region_id;
|
|
||||||
|
|
||||||
while (region_id >= new_parents.size())
|
|
||||||
{
|
|
||||||
new_parents.resize(new_parents.size() * 2);
|
|
||||||
new_populations.resize(new_parents.size());
|
|
||||||
types.resize(new_parents.size());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
new_parents[region_id] = parent_id;
|
|
||||||
new_populations[region_id] = population;
|
|
||||||
types[region_id] = type;
|
|
||||||
}
|
|
||||||
|
|
||||||
new_parents .resize(max_region_id + 1);
|
|
||||||
new_city .resize(max_region_id + 1);
|
|
||||||
new_country .resize(max_region_id + 1);
|
|
||||||
new_area .resize(max_region_id + 1);
|
|
||||||
new_district .resize(max_region_id + 1);
|
|
||||||
new_continent .resize(max_region_id + 1);
|
|
||||||
new_top_continent.resize(max_region_id + 1);
|
|
||||||
new_populations .resize(max_region_id + 1);
|
|
||||||
new_depths .resize(max_region_id + 1);
|
|
||||||
types .resize(max_region_id + 1);
|
|
||||||
|
|
||||||
/// пропишем города и страны для регионов
|
|
||||||
for (RegionID i = 0; i <= max_region_id; ++i)
|
|
||||||
{
|
|
||||||
if (types[i] == REGION_TYPE_CITY)
|
|
||||||
new_city[i] = i;
|
|
||||||
|
|
||||||
if (types[i] == REGION_TYPE_AREA)
|
|
||||||
new_area[i] = i;
|
|
||||||
|
|
||||||
if (types[i] == REGION_TYPE_DISTRICT)
|
|
||||||
new_district[i] = i;
|
|
||||||
|
|
||||||
if (types[i] == REGION_TYPE_COUNTRY)
|
|
||||||
new_country[i] = i;
|
|
||||||
|
|
||||||
if (types[i] == REGION_TYPE_CONTINENT)
|
|
||||||
{
|
|
||||||
new_continent[i] = i;
|
|
||||||
new_top_continent[i] = i;
|
|
||||||
}
|
|
||||||
|
|
||||||
RegionDepth depth = 0;
|
|
||||||
RegionID current = i;
|
|
||||||
while (true)
|
|
||||||
{
|
|
||||||
++depth;
|
|
||||||
|
|
||||||
if (depth == std::numeric_limits<RegionDepth>::max())
|
|
||||||
throw Poco::Exception("Logical error in regions hierarchy: region " + DB::toString(current) + " possible is inside infinite loop");
|
|
||||||
|
|
||||||
current = new_parents[current];
|
|
||||||
if (current == 0)
|
|
||||||
break;
|
|
||||||
|
|
||||||
if (current > max_region_id)
|
|
||||||
throw Poco::Exception("Logical error in regions hierarchy: region " + DB::toString(current) + " (specified as parent) doesn't exist");
|
|
||||||
|
|
||||||
if (types[current] == REGION_TYPE_CITY)
|
|
||||||
new_city[i] = current;
|
|
||||||
|
|
||||||
if (types[current] == REGION_TYPE_AREA)
|
|
||||||
new_area[i] = current;
|
|
||||||
|
|
||||||
if (types[current] == REGION_TYPE_DISTRICT)
|
|
||||||
new_district[i] = current;
|
|
||||||
|
|
||||||
if (types[current] == REGION_TYPE_COUNTRY)
|
|
||||||
new_country[i] = current;
|
|
||||||
|
|
||||||
if (types[current] == REGION_TYPE_CONTINENT)
|
|
||||||
{
|
|
||||||
if (!new_continent[i])
|
|
||||||
new_continent[i] = current;
|
|
||||||
new_top_continent[i] = current;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
new_depths[i] = depth;
|
|
||||||
}
|
|
||||||
|
|
||||||
parents.swap(new_parents);
|
|
||||||
country.swap(new_country);
|
|
||||||
city.swap(new_city);
|
|
||||||
area.swap(new_area);
|
|
||||||
district.swap(new_district);
|
|
||||||
continent.swap(new_continent);
|
|
||||||
top_continent.swap(new_top_continent);
|
|
||||||
populations.swap(new_populations);
|
|
||||||
depths.swap(new_depths);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
bool in(RegionID lhs, RegionID rhs) const
|
bool in(RegionID lhs, RegionID rhs) const
|
||||||
|
@ -1,23 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <sparsehash/dense_hash_map>
|
#include <string>
|
||||||
|
#include <vector>
|
||||||
#include <Poco/File.h>
|
|
||||||
#include <Poco/NumberParser.h>
|
|
||||||
#include <Poco/Util/Application.h>
|
|
||||||
#include <Poco/Exception.h>
|
#include <Poco/Exception.h>
|
||||||
|
|
||||||
#include <common/Common.h>
|
#include <common/Common.h>
|
||||||
#include <common/logger_useful.h>
|
|
||||||
|
|
||||||
#include <DB/Core/StringRef.h>
|
#include <DB/Core/StringRef.h>
|
||||||
|
|
||||||
#include <DB/IO/ReadHelpers.h>
|
|
||||||
#include <DB/IO/WriteHelpers.h>
|
|
||||||
#include <DB/IO/ReadBufferFromFile.h>
|
|
||||||
|
|
||||||
|
/** Класс, позволяющий узнавать по id региона его текстовое название на одном из поддерживаемых языков: ru, en, ua, by, kz, tr.
|
||||||
/** @brief Класс, позволяющий узнавать по id региона его текстовое название на одном из поддерживаемых языков: ru, en, ua, by, kz, tr.
|
|
||||||
*
|
*
|
||||||
* Информацию об именах регионов загружает из текстовых файлов с названиями следующего формата:
|
* Информацию об именах регионов загружает из текстовых файлов с названиями следующего формата:
|
||||||
* regions_names_xx.txt,
|
* regions_names_xx.txt,
|
||||||
@ -75,89 +65,13 @@ private:
|
|||||||
using StringRefsForLanguageID = std::vector<StringRefs>;
|
using StringRefsForLanguageID = std::vector<StringRefs>;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static constexpr auto required_key = "path_to_regions_names_files";
|
/** Перезагружает, при необходимости, имена регионов.
|
||||||
|
|
||||||
RegionsNames(const std::string & directory_ = Poco::Util::Application::instance().config().getString(required_key))
|
|
||||||
: directory(directory_)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/** @brief Перезагружает, при необходимости, имена регионов.
|
|
||||||
*/
|
*/
|
||||||
void reload()
|
void reload();
|
||||||
{
|
|
||||||
LOG_DEBUG(log, "Reloading regions names");
|
|
||||||
|
|
||||||
RegionID max_region_id = 0;
|
/// Has corresponding section in configuration file.
|
||||||
for (size_t language_id = 0; language_id < SUPPORTED_LANGUAGES_COUNT; ++language_id)
|
static bool isConfigured();
|
||||||
{
|
|
||||||
const std::string & language = getSupportedLanguages()[language_id];
|
|
||||||
std::string path = directory + "/regions_names_" + language + ".txt";
|
|
||||||
|
|
||||||
Poco::File file(path);
|
|
||||||
time_t new_modification_time = file.getLastModified().epochTime();
|
|
||||||
if (new_modification_time <= file_modification_times[language_id])
|
|
||||||
continue;
|
|
||||||
file_modification_times[language_id] = new_modification_time;
|
|
||||||
|
|
||||||
LOG_DEBUG(log, "Reloading regions names for language: " << language);
|
|
||||||
|
|
||||||
DB::ReadBufferFromFile in(path);
|
|
||||||
|
|
||||||
const size_t initial_size = 10000;
|
|
||||||
const size_t max_size = 1000000;
|
|
||||||
|
|
||||||
Chars new_chars;
|
|
||||||
StringRefs new_names_refs(initial_size, StringRef("", 0));
|
|
||||||
|
|
||||||
/// Выделим непрерывный кусок памяти, которого хватит для хранения всех имён.
|
|
||||||
new_chars.reserve(Poco::File(path).getSize());
|
|
||||||
|
|
||||||
while (!in.eof())
|
|
||||||
{
|
|
||||||
Int32 read_region_id;
|
|
||||||
std::string region_name;
|
|
||||||
|
|
||||||
DB::readIntText(read_region_id, in);
|
|
||||||
DB::assertChar('\t', in);
|
|
||||||
DB::readString(region_name, in);
|
|
||||||
DB::assertChar('\n', in);
|
|
||||||
|
|
||||||
if (read_region_id <= 0)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
RegionID region_id = read_region_id;
|
|
||||||
|
|
||||||
size_t old_size = new_chars.size();
|
|
||||||
|
|
||||||
if (new_chars.capacity() < old_size + region_name.length() + 1)
|
|
||||||
throw Poco::Exception("Logical error. Maybe size of file " + path + " is wrong.");
|
|
||||||
|
|
||||||
new_chars.resize(old_size + region_name.length() + 1);
|
|
||||||
memcpy(&new_chars[old_size], region_name.c_str(), region_name.length() + 1);
|
|
||||||
|
|
||||||
if (region_id > max_region_id)
|
|
||||||
{
|
|
||||||
max_region_id = region_id;
|
|
||||||
|
|
||||||
if (region_id > max_size)
|
|
||||||
throw DB::Exception("Region id is too large: " + DB::toString(region_id) + ", should be not more than " + DB::toString(max_size));
|
|
||||||
}
|
|
||||||
|
|
||||||
while (region_id >= new_names_refs.size())
|
|
||||||
new_names_refs.resize(new_names_refs.size() * 2, StringRef("", 0));
|
|
||||||
|
|
||||||
new_names_refs[region_id] = StringRef(&new_chars[old_size], region_name.length());
|
|
||||||
}
|
|
||||||
|
|
||||||
chars[language_id].swap(new_chars);
|
|
||||||
names_refs[language_id].swap(new_names_refs);
|
|
||||||
}
|
|
||||||
|
|
||||||
for (size_t language_id = 0; language_id < SUPPORTED_LANGUAGES_COUNT; ++language_id)
|
|
||||||
names_refs[language_id].resize(max_region_id + 1, StringRef("", 0));
|
|
||||||
}
|
|
||||||
|
|
||||||
StringRef getRegionName(RegionID region_id, Language language = Language::RU) const
|
StringRef getRegionName(RegionID region_id, Language language = Language::RU) const
|
||||||
{
|
{
|
||||||
@ -192,24 +106,10 @@ public:
|
|||||||
throw Poco::Exception("Unsupported language for region name. Supported languages are: " + dumpSupportedLanguagesNames() + ".");
|
throw Poco::Exception("Unsupported language for region name. Supported languages are: " + dumpSupportedLanguagesNames() + ".");
|
||||||
}
|
}
|
||||||
|
|
||||||
static std::string dumpSupportedLanguagesNames()
|
|
||||||
{
|
|
||||||
std::string res = "";
|
|
||||||
for (size_t i = 0; i < LANGUAGE_ALIASES_COUNT; ++i)
|
|
||||||
{
|
|
||||||
if (i > 0)
|
|
||||||
res += ", ";
|
|
||||||
res += '\'';
|
|
||||||
res += getLanguageAliases()[i].name;
|
|
||||||
res += '\'';
|
|
||||||
}
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const std::string directory;
|
static std::string dumpSupportedLanguagesNames();
|
||||||
|
|
||||||
ModificationTimes file_modification_times = ModificationTimes(SUPPORTED_LANGUAGES_COUNT);
|
ModificationTimes file_modification_times = ModificationTimes(SUPPORTED_LANGUAGES_COUNT);
|
||||||
Logger * log = &Logger::get("RegionsNames");
|
|
||||||
|
|
||||||
/// Байты имен для каждого языка, уложенные подряд, разделенные нулями
|
/// Байты имен для каждого языка, уложенные подряд, разделенные нулями
|
||||||
CharsForLanguageID chars = CharsForLanguageID(SUPPORTED_LANGUAGES_COUNT);
|
CharsForLanguageID chars = CharsForLanguageID(SUPPORTED_LANGUAGES_COUNT);
|
||||||
|
@ -1,11 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <string.h>
|
|
||||||
|
|
||||||
#include <common/logger_useful.h>
|
|
||||||
#include <common/singleton.h>
|
#include <common/singleton.h>
|
||||||
|
#include <common/Common.h>
|
||||||
#include <mysqlxx/PoolWithFailover.h>
|
|
||||||
|
|
||||||
|
|
||||||
/** @brief Класс, позволяющий узнавать, принадлежит ли поисковая система или операционная система
|
/** @brief Класс, позволяющий узнавать, принадлежит ли поисковая система или операционная система
|
||||||
@ -15,57 +11,14 @@
|
|||||||
class TechDataHierarchy
|
class TechDataHierarchy
|
||||||
{
|
{
|
||||||
private:
|
private:
|
||||||
Logger * log;
|
UInt8 os_parent[256] {};
|
||||||
|
UInt8 se_parent[256] {};
|
||||||
UInt8 os_parent[256];
|
|
||||||
UInt8 se_parent[256];
|
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static constexpr auto required_key = "mysql_metrica";
|
void reload();
|
||||||
|
|
||||||
TechDataHierarchy()
|
/// Has corresponding section in configuration file.
|
||||||
: log(&Logger::get("TechDataHierarchy"))
|
static bool isConfigured();
|
||||||
{
|
|
||||||
LOG_DEBUG(log, "Loading tech data hierarchy.");
|
|
||||||
|
|
||||||
memset(os_parent, 0, sizeof(os_parent));
|
|
||||||
memset(se_parent, 0, sizeof(se_parent));
|
|
||||||
|
|
||||||
mysqlxx::PoolWithFailover pool(required_key);
|
|
||||||
mysqlxx::Pool::Entry conn = pool.Get();
|
|
||||||
|
|
||||||
{
|
|
||||||
mysqlxx::Query q = conn->query("SELECT Id, COALESCE(Parent_Id, 0) FROM OS2");
|
|
||||||
LOG_TRACE(log, q.str());
|
|
||||||
mysqlxx::UseQueryResult res = q.use();
|
|
||||||
while (mysqlxx::Row row = res.fetch())
|
|
||||||
{
|
|
||||||
UInt64 child = row[0].getUInt();
|
|
||||||
UInt64 parent = row[1].getUInt();
|
|
||||||
|
|
||||||
if (child > 255 || parent > 255)
|
|
||||||
throw Poco::Exception("Too large OS id (> 255).");
|
|
||||||
|
|
||||||
os_parent[child] = parent;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
{
|
|
||||||
mysqlxx::Query q = conn->query("SELECT Id, COALESCE(ParentId, 0) FROM SearchEngines");
|
|
||||||
LOG_TRACE(log, q.str());
|
|
||||||
mysqlxx::UseQueryResult res = q.use();
|
|
||||||
while (mysqlxx::Row row = res.fetch())
|
|
||||||
{
|
|
||||||
UInt64 child = row[0].getUInt();
|
|
||||||
UInt64 parent = row[1].getUInt();
|
|
||||||
|
|
||||||
if (child > 255 || parent > 255)
|
|
||||||
throw Poco::Exception("Too large search engine id (> 255).");
|
|
||||||
|
|
||||||
se_parent[child] = parent;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/// Отношение "принадлежит".
|
/// Отношение "принадлежит".
|
||||||
|
@ -3,10 +3,13 @@
|
|||||||
#include <DB/Dictionaries/IDictionarySource.h>
|
#include <DB/Dictionaries/IDictionarySource.h>
|
||||||
#include <DB/Dictionaries/DictionaryStructure.h>
|
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
||||||
/// Allows loading dictionaries from executable
|
/// Allows loading dictionaries from executable
|
||||||
class ExecutableDictionarySource final : public IDictionarySource
|
class ExecutableDictionarySource final : public IDictionarySource
|
||||||
{
|
{
|
||||||
@ -16,8 +19,7 @@ public:
|
|||||||
const Poco::Util::AbstractConfiguration & config,
|
const Poco::Util::AbstractConfiguration & config,
|
||||||
const std::string & config_prefix,
|
const std::string & config_prefix,
|
||||||
Block & sample_block,
|
Block & sample_block,
|
||||||
const Context & context
|
const Context & context);
|
||||||
);
|
|
||||||
|
|
||||||
ExecutableDictionarySource(const ExecutableDictionarySource & other);
|
ExecutableDictionarySource(const ExecutableDictionarySource & other);
|
||||||
|
|
||||||
@ -37,7 +39,7 @@ public:
|
|||||||
std::string toString() const override;
|
std::string toString() const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
Logger * log = &Logger::get("ExecutableDictionarySource");
|
Poco::Logger * log;
|
||||||
|
|
||||||
const DictionaryStructure dict_struct;
|
const DictionaryStructure dict_struct;
|
||||||
const std::string command;
|
const std::string command;
|
||||||
|
@ -3,10 +3,13 @@
|
|||||||
#include <DB/Dictionaries/IDictionarySource.h>
|
#include <DB/Dictionaries/IDictionarySource.h>
|
||||||
#include <DB/Dictionaries/DictionaryStructure.h>
|
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
||||||
/// Allows loading dictionaries from executable
|
/// Allows loading dictionaries from executable
|
||||||
class HTTPDictionarySource final : public IDictionarySource
|
class HTTPDictionarySource final : public IDictionarySource
|
||||||
{
|
{
|
||||||
@ -35,7 +38,7 @@ public:
|
|||||||
std::string toString() const override;
|
std::string toString() const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
Logger * log = &Logger::get("HTTPDictionarySource");
|
Poco::Logger * log;
|
||||||
|
|
||||||
LocalDateTime getLastModification() const;
|
LocalDateTime getLastModification() const;
|
||||||
|
|
||||||
|
@ -1,14 +1,16 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Dictionaries/IDictionarySource.h>
|
#include <DB/Dictionaries/IDictionarySource.h>
|
||||||
#include <DB/Dictionaries/MySQLBlockInputStream.h>
|
|
||||||
#include <DB/Dictionaries/ExternalQueryBuilder.h>
|
#include <DB/Dictionaries/ExternalQueryBuilder.h>
|
||||||
#include <DB/Dictionaries/DictionaryStructure.h>
|
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||||
#include <ext/range.hpp>
|
#include <ext/range.hpp>
|
||||||
#include <mysqlxx/Pool.h>
|
#include <mysqlxx/PoolWithFailover.h>
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -40,7 +42,7 @@ public:
|
|||||||
std::string toString() const override;
|
std::string toString() const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
Logger * log = &Logger::get("MySQLDictionarySource");
|
Poco::Logger * log;
|
||||||
|
|
||||||
static std::string quoteForLike(const std::string s);
|
static std::string quoteForLike(const std::string s);
|
||||||
|
|
||||||
|
@ -16,6 +16,8 @@ namespace Poco
|
|||||||
{
|
{
|
||||||
class AbstractConfiguration;
|
class AbstractConfiguration;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class Logger;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -50,7 +52,7 @@ public:
|
|||||||
std::string toString() const override;
|
std::string toString() const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
Logger * log = &Logger::get("ODBCDictionarySource");
|
Poco::Logger * log;
|
||||||
|
|
||||||
const DictionaryStructure dict_struct;
|
const DictionaryStructure dict_struct;
|
||||||
const std::string db;
|
const std::string db;
|
||||||
|
@ -21,8 +21,6 @@
|
|||||||
#include <DB/Functions/FunctionsConditional.h>
|
#include <DB/Functions/FunctionsConditional.h>
|
||||||
#include <DB/Functions/FunctionsConversion.h>
|
#include <DB/Functions/FunctionsConversion.h>
|
||||||
#include <DB/Functions/Conditional/getArrayType.h>
|
#include <DB/Functions/Conditional/getArrayType.h>
|
||||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
|
||||||
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
|
||||||
#include <DB/Parsers/ExpressionListParsers.h>
|
#include <DB/Parsers/ExpressionListParsers.h>
|
||||||
#include <DB/Parsers/parseQuery.h>
|
#include <DB/Parsers/parseQuery.h>
|
||||||
#include <DB/Parsers/ASTExpressionList.h>
|
#include <DB/Parsers/ASTExpressionList.h>
|
||||||
@ -1432,6 +1430,9 @@ private:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
class IAggregateFunction;
|
||||||
|
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
|
||||||
|
|
||||||
/** Применяет к массиву агрегатную функцию и возвращает её результат.
|
/** Применяет к массиву агрегатную функцию и возвращает её результат.
|
||||||
* Также может быть применена к нескольким массивам одинаковых размеров, если агрегатная функция принимает несколько аргументов.
|
* Также может быть применена к нескольким массивам одинаковых размеров, если агрегатная функция принимает несколько аргументов.
|
||||||
*/
|
*/
|
||||||
|
@ -13,10 +13,14 @@
|
|||||||
#include <DB/Columns/ColumnTuple.h>
|
#include <DB/Columns/ColumnTuple.h>
|
||||||
|
|
||||||
#include <DB/Interpreters/Context.h>
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <DB/Interpreters/Dictionaries.h>
|
#include <DB/Interpreters/EmbeddedDictionaries.h>
|
||||||
#include <DB/Interpreters/ExternalDictionaries.h>
|
#include <DB/Interpreters/ExternalDictionaries.h>
|
||||||
|
|
||||||
#include <DB/Functions/IFunction.h>
|
#include <DB/Functions/IFunction.h>
|
||||||
|
#include <DB/Dictionaries/Embedded/RegionsHierarchy.h>
|
||||||
|
#include <DB/Dictionaries/Embedded/RegionsHierarchies.h>
|
||||||
|
#include <DB/Dictionaries/Embedded/RegionsNames.h>
|
||||||
|
#include <DB/Dictionaries/Embedded/TechDataHierarchy.h>
|
||||||
#include <DB/Dictionaries/FlatDictionary.h>
|
#include <DB/Dictionaries/FlatDictionary.h>
|
||||||
#include <DB/Dictionaries/HashedDictionary.h>
|
#include <DB/Dictionaries/HashedDictionary.h>
|
||||||
#include <DB/Dictionaries/CacheDictionary.h>
|
#include <DB/Dictionaries/CacheDictionary.h>
|
||||||
@ -553,7 +557,7 @@ struct FunctionRegionToCity :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -562,7 +566,7 @@ struct FunctionRegionToArea :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -571,7 +575,7 @@ struct FunctionRegionToDistrict :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -580,7 +584,7 @@ struct FunctionRegionToCountry :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -589,7 +593,7 @@ struct FunctionRegionToContinent :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -598,7 +602,7 @@ struct FunctionRegionToTopContinent :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -607,7 +611,7 @@ struct FunctionRegionToPopulation :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -616,7 +620,7 @@ struct FunctionOSToRoot :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getTechDataHierarchy());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -625,7 +629,7 @@ struct FunctionSEToRoot :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getTechDataHierarchy());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -634,7 +638,7 @@ struct FunctionRegionIn :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -643,7 +647,7 @@ struct FunctionOSIn :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getTechDataHierarchy());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -652,7 +656,7 @@ struct FunctionSEIn :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getTechDataHierarchy());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -661,7 +665,7 @@ struct FunctionRegionHierarchy :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getRegionsHierarchies());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -670,7 +674,7 @@ struct FunctionOSHierarchy :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getTechDataHierarchy());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -679,7 +683,7 @@ struct FunctionSEHierarchy :
|
|||||||
{
|
{
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<base_type>(context.getDictionaries().getTechDataHierarchy());
|
return std::make_shared<base_type>(context.getEmbeddedDictionaries().getTechDataHierarchy());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -691,7 +695,7 @@ public:
|
|||||||
static constexpr auto name = "regionToName";
|
static constexpr auto name = "regionToName";
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<FunctionRegionToName>(context.getDictionaries().getRegionsNames());
|
return std::make_shared<FunctionRegionToName>(context.getEmbeddedDictionaries().getRegionsNames());
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
101
dbms/include/DB/IO/MySQLxxHelpers.h
Normal file
101
dbms/include/DB/IO/MySQLxxHelpers.h
Normal file
@ -0,0 +1,101 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <mysqlxx/Row.h>
|
||||||
|
#include <mysqlxx/Null.h>
|
||||||
|
#include <mysqlxx/Manip.h>
|
||||||
|
|
||||||
|
#include <DB/Core/Field.h>
|
||||||
|
#include <DB/Core/FieldVisitors.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
|
/// This is for Yandex.Metrica code.
|
||||||
|
|
||||||
|
namespace mysqlxx
|
||||||
|
{
|
||||||
|
inline std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Array & value)
|
||||||
|
{
|
||||||
|
return res.ostr << DB::applyVisitor(DB::FieldVisitorToString(), DB::Field(value));
|
||||||
|
}
|
||||||
|
|
||||||
|
inline std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Array & value)
|
||||||
|
{
|
||||||
|
throw Poco::Exception("Cannot quote Array with mysqlxx::quote.");
|
||||||
|
}
|
||||||
|
|
||||||
|
inline std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Array & value)
|
||||||
|
{
|
||||||
|
throw Poco::Exception("Cannot unescape Array with mysqlxx::unescape.");
|
||||||
|
}
|
||||||
|
|
||||||
|
inline std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Array & value)
|
||||||
|
{
|
||||||
|
throw Poco::Exception("Cannot unquote Array with mysqlxx::unquote.");
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
inline std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Tuple & value)
|
||||||
|
{
|
||||||
|
return res.ostr << DB::applyVisitor(DB::FieldVisitorToString(), DB::Field(value));
|
||||||
|
}
|
||||||
|
|
||||||
|
inline std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Tuple & value)
|
||||||
|
{
|
||||||
|
throw Poco::Exception("Cannot quote Tuple with mysqlxx::quote.");
|
||||||
|
}
|
||||||
|
|
||||||
|
inline std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Tuple & value)
|
||||||
|
{
|
||||||
|
throw Poco::Exception("Cannot unescape Tuple with mysqlxx::unescape.");
|
||||||
|
}
|
||||||
|
|
||||||
|
inline std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Tuple & value)
|
||||||
|
{
|
||||||
|
throw Poco::Exception("Cannot unquote Tuple with mysqlxx::unquote.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Вывести mysqlxx::Row в tab-separated виде
|
||||||
|
inline void writeEscapedRow(const mysqlxx::Row & row, WriteBuffer & buf)
|
||||||
|
{
|
||||||
|
for (size_t i = 0; i < row.size(); ++i)
|
||||||
|
{
|
||||||
|
if (i != 0)
|
||||||
|
buf.write('\t');
|
||||||
|
|
||||||
|
if (unlikely(row[i].isNull()))
|
||||||
|
{
|
||||||
|
buf.write("\\N", 2);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
writeAnyEscapedString<'\''>(row[i].data(), row[i].data() + row[i].length(), buf);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
inline void writeText(const mysqlxx::Null<T> & x, WriteBuffer & buf)
|
||||||
|
{
|
||||||
|
if (x.isNull())
|
||||||
|
writeCString("\\N", buf);
|
||||||
|
else
|
||||||
|
writeText(static_cast<const T &>(x), buf);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
inline void writeQuoted(const mysqlxx::Null<T> & x, WriteBuffer & buf)
|
||||||
|
{
|
||||||
|
if (x.isNull())
|
||||||
|
writeCString("NULL", buf);
|
||||||
|
else
|
||||||
|
writeText(static_cast<const T &>(x), buf);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -7,11 +7,10 @@
|
|||||||
|
|
||||||
#include <common/Common.h>
|
#include <common/Common.h>
|
||||||
#include <common/DateLUT.h>
|
#include <common/DateLUT.h>
|
||||||
|
#include <common/LocalDate.h>
|
||||||
|
#include <common/LocalDateTime.h>
|
||||||
#include <common/find_first_symbols.h>
|
#include <common/find_first_symbols.h>
|
||||||
|
|
||||||
#include <mysqlxx/Row.h>
|
|
||||||
#include <mysqlxx/Null.h>
|
|
||||||
|
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
#include <DB/Common/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Common/StringUtils.h>
|
#include <DB/Common/StringUtils.h>
|
||||||
@ -580,25 +579,6 @@ inline void writeDateTimeText(LocalDateTime datetime, WriteBuffer & buf)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/// Вывести mysqlxx::Row в tab-separated виде
|
|
||||||
inline void writeEscapedRow(const mysqlxx::Row & row, WriteBuffer & buf)
|
|
||||||
{
|
|
||||||
for (size_t i = 0; i < row.size(); ++i)
|
|
||||||
{
|
|
||||||
if (i != 0)
|
|
||||||
buf.write('\t');
|
|
||||||
|
|
||||||
if (unlikely(row[i].isNull()))
|
|
||||||
{
|
|
||||||
buf.write("\\N", 2);
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
writeAnyEscapedString<'\''>(row[i].data(), row[i].data() + row[i].length(), buf);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/// Методы вывода в бинарном виде
|
/// Методы вывода в бинарном виде
|
||||||
inline void writeBinary(const UInt8 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
|
inline void writeBinary(const UInt8 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
|
||||||
inline void writeBinary(const UInt16 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
|
inline void writeBinary(const UInt16 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
|
||||||
@ -649,15 +629,6 @@ inline void writeText(const VisitID_t & x, WriteBuffer & buf) { writeIntText(st
|
|||||||
inline void writeText(const LocalDate & x, WriteBuffer & buf) { writeDateText(x, buf); }
|
inline void writeText(const LocalDate & x, WriteBuffer & buf) { writeDateText(x, buf); }
|
||||||
inline void writeText(const LocalDateTime & x, WriteBuffer & buf) { writeDateTimeText(x, buf); }
|
inline void writeText(const LocalDateTime & x, WriteBuffer & buf) { writeDateTimeText(x, buf); }
|
||||||
|
|
||||||
template<typename T>
|
|
||||||
inline void writeText(const mysqlxx::Null<T> & x, WriteBuffer & buf)
|
|
||||||
{
|
|
||||||
if (x.isNull())
|
|
||||||
writeCString("\\N", buf);
|
|
||||||
else
|
|
||||||
writeText(static_cast<const T &>(x), buf);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/// Строки, даты, даты-с-временем - в одинарных кавычках с C-style эскейпингом. Числа - без.
|
/// Строки, даты, даты-с-временем - в одинарных кавычках с C-style эскейпингом. Числа - без.
|
||||||
inline void writeQuoted(const UInt8 & x, WriteBuffer & buf) { writeText(x, buf); }
|
inline void writeQuoted(const UInt8 & x, WriteBuffer & buf) { writeText(x, buf); }
|
||||||
@ -692,15 +663,6 @@ inline void writeQuoted(const LocalDateTime & x, WriteBuffer & buf)
|
|||||||
writeChar('\'', buf);
|
writeChar('\'', buf);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
|
||||||
inline void writeQuoted(const mysqlxx::Null<T> & x, WriteBuffer & buf)
|
|
||||||
{
|
|
||||||
if (x.isNull())
|
|
||||||
writeCString("NULL", buf);
|
|
||||||
else
|
|
||||||
writeText(static_cast<const T &>(x), buf);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/// Строки, даты, даты-с-временем - в двойных кавычках с C-style эскейпингом. Числа - без.
|
/// Строки, даты, даты-с-временем - в двойных кавычках с C-style эскейпингом. Числа - без.
|
||||||
inline void writeDoubleQuoted(const UInt8 & x, WriteBuffer & buf) { writeText(x, buf); }
|
inline void writeDoubleQuoted(const UInt8 & x, WriteBuffer & buf) { writeText(x, buf); }
|
||||||
|
@ -36,6 +36,8 @@ namespace ErrorCodes
|
|||||||
extern const int UNKNOWN_AGGREGATED_DATA_VARIANT;
|
extern const int UNKNOWN_AGGREGATED_DATA_VARIANT;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class IBlockOutputStream;
|
||||||
|
|
||||||
|
|
||||||
/** Разные структуры данных, которые могут использоваться для агрегации
|
/** Разные структуры данных, которые могут использоваться для агрегации
|
||||||
* Для эффективности, сами данные для агрегации кладутся в пул.
|
* Для эффективности, сами данные для агрегации кладутся в пул.
|
||||||
|
@ -7,11 +7,16 @@
|
|||||||
#include <DB/Core/NamesAndTypes.h>
|
#include <DB/Core/NamesAndTypes.h>
|
||||||
#include <DB/Interpreters/Settings.h>
|
#include <DB/Interpreters/Settings.h>
|
||||||
#include <DB/Interpreters/ClientInfo.h>
|
#include <DB/Interpreters/ClientInfo.h>
|
||||||
#include <DB/Storages/IStorage.h>
|
|
||||||
#include <DB/IO/CompressedStream.h>
|
#include <DB/IO/CompressedStream.h>
|
||||||
|
|
||||||
#include <Poco/Net/IPAddress.h>
|
|
||||||
|
|
||||||
|
namespace Poco
|
||||||
|
{
|
||||||
|
namespace Net
|
||||||
|
{
|
||||||
|
class IPAddress;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
namespace zkutil
|
namespace zkutil
|
||||||
{
|
{
|
||||||
@ -26,7 +31,7 @@ struct ContextShared;
|
|||||||
class QuotaForIntervals;
|
class QuotaForIntervals;
|
||||||
class TableFunctionFactory;
|
class TableFunctionFactory;
|
||||||
class AggregateFunctionFactory;
|
class AggregateFunctionFactory;
|
||||||
class Dictionaries;
|
class EmbeddedDictionaries;
|
||||||
class ExternalDictionaries;
|
class ExternalDictionaries;
|
||||||
class InterserverIOHandler;
|
class InterserverIOHandler;
|
||||||
class BackgroundProcessingPool;
|
class BackgroundProcessingPool;
|
||||||
@ -45,6 +50,16 @@ class QueryLog;
|
|||||||
struct MergeTreeSettings;
|
struct MergeTreeSettings;
|
||||||
class IDatabase;
|
class IDatabase;
|
||||||
class DDLGuard;
|
class DDLGuard;
|
||||||
|
class IStorage;
|
||||||
|
using StoragePtr = std::shared_ptr<IStorage>;
|
||||||
|
using Tables = std::map<String, StoragePtr>;
|
||||||
|
class IAST;
|
||||||
|
using ASTPtr = std::shared_ptr<IAST>;
|
||||||
|
class IBlockInputStream;
|
||||||
|
class IBlockOutputStream;
|
||||||
|
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
||||||
|
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
|
||||||
|
class Block;
|
||||||
|
|
||||||
|
|
||||||
/// (имя базы данных, имя таблицы)
|
/// (имя базы данных, имя таблицы)
|
||||||
@ -172,9 +187,9 @@ public:
|
|||||||
|
|
||||||
const TableFunctionFactory & getTableFunctionFactory() const;
|
const TableFunctionFactory & getTableFunctionFactory() const;
|
||||||
const AggregateFunctionFactory & getAggregateFunctionFactory() const;
|
const AggregateFunctionFactory & getAggregateFunctionFactory() const;
|
||||||
const Dictionaries & getDictionaries() const;
|
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
|
||||||
const ExternalDictionaries & getExternalDictionaries() const;
|
const ExternalDictionaries & getExternalDictionaries() const;
|
||||||
void tryCreateDictionaries() const;
|
void tryCreateEmbeddedDictionaries() const;
|
||||||
void tryCreateExternalDictionaries() const;
|
void tryCreateExternalDictionaries() const;
|
||||||
|
|
||||||
/// Форматы ввода-вывода.
|
/// Форматы ввода-вывода.
|
||||||
@ -294,7 +309,7 @@ private:
|
|||||||
*/
|
*/
|
||||||
void checkDatabaseAccessRights(const std::string & database_name) const;
|
void checkDatabaseAccessRights(const std::string & database_name) const;
|
||||||
|
|
||||||
const Dictionaries & getDictionariesImpl(bool throw_on_error) const;
|
const EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const;
|
||||||
const ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const;
|
const ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const;
|
||||||
|
|
||||||
StoragePtr getTableImpl(const String & database_name, const String & table_name, Exception * exception) const;
|
StoragePtr getTableImpl(const String & database_name, const String & table_name, Exception * exception) const;
|
||||||
|
@ -1,178 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <thread>
|
|
||||||
#include <common/MultiVersion.h>
|
|
||||||
#include <common/logger_useful.h>
|
|
||||||
#include <DB/Dictionaries/Embedded/RegionsHierarchies.h>
|
|
||||||
#include <DB/Dictionaries/Embedded/TechDataHierarchy.h>
|
|
||||||
#include <DB/Dictionaries/Embedded/RegionsNames.h>
|
|
||||||
#include <DB/Common/setThreadName.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
class Context;
|
|
||||||
|
|
||||||
/// Metrica's Dictionaries which can be used in functions.
|
|
||||||
|
|
||||||
class Dictionaries
|
|
||||||
{
|
|
||||||
private:
|
|
||||||
MultiVersion<RegionsHierarchies> regions_hierarchies;
|
|
||||||
MultiVersion<TechDataHierarchy> tech_data_hierarchy;
|
|
||||||
MultiVersion<RegionsNames> regions_names;
|
|
||||||
|
|
||||||
/// Directories' updating periodicity (in seconds).
|
|
||||||
int reload_period;
|
|
||||||
int cur_reload_period = 1;
|
|
||||||
bool is_fast_start_stage = true;
|
|
||||||
|
|
||||||
std::thread reloading_thread;
|
|
||||||
Poco::Event destroy;
|
|
||||||
|
|
||||||
Logger * log = &Logger::get("Dictionaries");
|
|
||||||
|
|
||||||
|
|
||||||
void handleException(const bool throw_on_error) const
|
|
||||||
{
|
|
||||||
const auto exception_ptr = std::current_exception();
|
|
||||||
|
|
||||||
tryLogCurrentException(log, "Cannot load dictionary! You must resolve this manually.");
|
|
||||||
|
|
||||||
if (throw_on_error)
|
|
||||||
std::rethrow_exception(exception_ptr);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/// Updates dictionaries.
|
|
||||||
bool reloadImpl(const bool throw_on_error)
|
|
||||||
{
|
|
||||||
/** Если не удаётся обновить справочники, то несмотря на это, не кидаем исключение (используем старые справочники).
|
|
||||||
* Если старых корректных справочников нет, то при использовании функций, которые от них зависят,
|
|
||||||
* будет кидаться исключение.
|
|
||||||
* Производится попытка загрузить каждый справочник по-отдельности.
|
|
||||||
*/
|
|
||||||
|
|
||||||
LOG_INFO(log, "Loading dictionaries.");
|
|
||||||
|
|
||||||
auto & config = Poco::Util::Application::instance().config();
|
|
||||||
|
|
||||||
bool was_exception = false;
|
|
||||||
|
|
||||||
if (config.has(TechDataHierarchy::required_key) && (!is_fast_start_stage || !tech_data_hierarchy.get()))
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
auto new_tech_data_hierarchy = std::make_unique<TechDataHierarchy>();
|
|
||||||
tech_data_hierarchy.set(new_tech_data_hierarchy.release());
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
handleException(throw_on_error);
|
|
||||||
was_exception = true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (config.has(RegionsHierarchies::required_key) && (!is_fast_start_stage || !regions_hierarchies.get()))
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
auto new_regions_hierarchies = std::make_unique<RegionsHierarchies>();
|
|
||||||
new_regions_hierarchies->reload();
|
|
||||||
regions_hierarchies.set(new_regions_hierarchies.release());
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
handleException(throw_on_error);
|
|
||||||
was_exception = true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (config.has(RegionsNames::required_key) && (!is_fast_start_stage || !regions_names.get()))
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
auto new_regions_names = std::make_unique<RegionsNames>();
|
|
||||||
new_regions_names->reload();
|
|
||||||
regions_names.set(new_regions_names.release());
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
handleException(throw_on_error);
|
|
||||||
was_exception = true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!was_exception)
|
|
||||||
LOG_INFO(log, "Loaded dictionaries.");
|
|
||||||
|
|
||||||
return !was_exception;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/** Updates directories (dictionaries) every reload_period seconds.
|
|
||||||
* If all dictionaries are not loaded at least once, try reload them with exponential delay (1, 2, ... reload_period).
|
|
||||||
* If all dictionaries are loaded, update them using constant reload_period delay.
|
|
||||||
*/
|
|
||||||
void reloadPeriodically()
|
|
||||||
{
|
|
||||||
setThreadName("DictReload");
|
|
||||||
|
|
||||||
while (true)
|
|
||||||
{
|
|
||||||
if (destroy.tryWait(cur_reload_period * 1000))
|
|
||||||
return;
|
|
||||||
|
|
||||||
if (reloadImpl(false))
|
|
||||||
{
|
|
||||||
/// Success
|
|
||||||
cur_reload_period = reload_period;
|
|
||||||
is_fast_start_stage = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (is_fast_start_stage)
|
|
||||||
{
|
|
||||||
cur_reload_period = std::min(reload_period, 2 * cur_reload_period); /// exponentially increase delay
|
|
||||||
is_fast_start_stage = cur_reload_period < reload_period; /// leave fast start state
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public:
|
|
||||||
/// Every reload_period seconds directories are updated inside a separate thread.
|
|
||||||
Dictionaries(const bool throw_on_error, const int reload_period_)
|
|
||||||
: reload_period(reload_period_)
|
|
||||||
{
|
|
||||||
reloadImpl(throw_on_error);
|
|
||||||
reloading_thread = std::thread([this] { reloadPeriodically(); });
|
|
||||||
}
|
|
||||||
|
|
||||||
Dictionaries(const bool throw_on_error)
|
|
||||||
: Dictionaries(throw_on_error,
|
|
||||||
Poco::Util::Application::instance().config().getInt("builtin_dictionaries_reload_interval", 3600))
|
|
||||||
{}
|
|
||||||
|
|
||||||
~Dictionaries()
|
|
||||||
{
|
|
||||||
destroy.set();
|
|
||||||
reloading_thread.join();
|
|
||||||
}
|
|
||||||
|
|
||||||
MultiVersion<RegionsHierarchies>::Version getRegionsHierarchies() const
|
|
||||||
{
|
|
||||||
return regions_hierarchies.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
MultiVersion<TechDataHierarchy>::Version getTechDataHierarchy() const
|
|
||||||
{
|
|
||||||
return tech_data_hierarchy.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
MultiVersion<RegionsNames>::Version getRegionsNames() const
|
|
||||||
{
|
|
||||||
return regions_names.get();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
80
dbms/include/DB/Interpreters/EmbeddedDictionaries.h
Normal file
80
dbms/include/DB/Interpreters/EmbeddedDictionaries.h
Normal file
@ -0,0 +1,80 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <thread>
|
||||||
|
#include <common/MultiVersion.h>
|
||||||
|
#include <Poco/Event.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
|
class RegionsHierarchies;
|
||||||
|
class TechDataHierarchy;
|
||||||
|
class RegionsNames;
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class Context;
|
||||||
|
|
||||||
|
|
||||||
|
/// Metrica's Dictionaries which can be used in functions.
|
||||||
|
|
||||||
|
class EmbeddedDictionaries
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
MultiVersion<RegionsHierarchies> regions_hierarchies;
|
||||||
|
MultiVersion<TechDataHierarchy> tech_data_hierarchy;
|
||||||
|
MultiVersion<RegionsNames> regions_names;
|
||||||
|
|
||||||
|
/// Directories' updating periodicity (in seconds).
|
||||||
|
int reload_period;
|
||||||
|
int cur_reload_period = 1;
|
||||||
|
bool is_fast_start_stage = true;
|
||||||
|
|
||||||
|
std::thread reloading_thread;
|
||||||
|
Poco::Event destroy;
|
||||||
|
|
||||||
|
Poco::Logger * log;
|
||||||
|
|
||||||
|
|
||||||
|
void handleException(const bool throw_on_error) const;
|
||||||
|
|
||||||
|
/// Updates dictionaries.
|
||||||
|
bool reloadImpl(const bool throw_on_error);
|
||||||
|
|
||||||
|
/** Updates directories (dictionaries) every reload_period seconds.
|
||||||
|
* If all dictionaries are not loaded at least once, try reload them with exponential delay (1, 2, ... reload_period).
|
||||||
|
* If all dictionaries are loaded, update them using constant reload_period delay.
|
||||||
|
*/
|
||||||
|
void reloadPeriodically();
|
||||||
|
|
||||||
|
template <typename Dictionary>
|
||||||
|
bool reloadDictionary(MultiVersion<Dictionary> & dictionary, const bool throw_on_error);
|
||||||
|
|
||||||
|
public:
|
||||||
|
/// Every reload_period seconds directories are updated inside a separate thread.
|
||||||
|
EmbeddedDictionaries(const bool throw_on_error, const int reload_period_);
|
||||||
|
|
||||||
|
EmbeddedDictionaries(const bool throw_on_error);
|
||||||
|
|
||||||
|
~EmbeddedDictionaries();
|
||||||
|
|
||||||
|
|
||||||
|
MultiVersion<RegionsHierarchies>::Version getRegionsHierarchies() const
|
||||||
|
{
|
||||||
|
return regions_hierarchies.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
MultiVersion<TechDataHierarchy>::Version getTechDataHierarchy() const
|
||||||
|
{
|
||||||
|
return tech_data_hierarchy.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
MultiVersion<RegionsNames>::Version getRegionsNames() const
|
||||||
|
{
|
||||||
|
return regions_names.get();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -4,6 +4,8 @@
|
|||||||
#include <DB/Storages/AlterCommands.h>
|
#include <DB/Storages/AlterCommands.h>
|
||||||
#include <DB/Interpreters/Context.h>
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <DB/Interpreters/IInterpreter.h>
|
#include <DB/Interpreters/IInterpreter.h>
|
||||||
|
#include <DB/Parsers/ASTAlterQuery.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <DB/Core/QueryProcessingStage.h>
|
||||||
#include <DB/Interpreters/IInterpreter.h>
|
#include <DB/Interpreters/IInterpreter.h>
|
||||||
|
|
||||||
|
|
||||||
@ -7,6 +8,9 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
|
class IAST;
|
||||||
|
using ASTPtr = std::shared_ptr<IAST>;
|
||||||
|
|
||||||
|
|
||||||
class InterpreterFactory
|
class InterpreterFactory
|
||||||
{
|
{
|
||||||
|
@ -6,6 +6,9 @@
|
|||||||
#include <DB/Interpreters/ExpressionActions.h>
|
#include <DB/Interpreters/ExpressionActions.h>
|
||||||
#include <DB/DataStreams/IBlockInputStream.h>
|
#include <DB/DataStreams/IBlockInputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -196,12 +199,12 @@ private:
|
|||||||
|
|
||||||
/// Таблица, откуда читать данные, если не подзапрос.
|
/// Таблица, откуда читать данные, если не подзапрос.
|
||||||
StoragePtr storage;
|
StoragePtr storage;
|
||||||
IStorage::TableStructureReadLockPtr table_lock;
|
TableStructureReadLockPtr table_lock;
|
||||||
|
|
||||||
/// Выполнить объединение потоков внутри запроса SELECT?
|
/// Выполнить объединение потоков внутри запроса SELECT?
|
||||||
bool union_within_single_query = false;
|
bool union_within_single_query = false;
|
||||||
|
|
||||||
Logger * log;
|
Poco::Logger * log;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,10 +1,11 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <common/logger_useful.h>
|
#include <Poco/RWLock.h>
|
||||||
|
|
||||||
#include <DB/Parsers/ASTTablesInSelectQuery.h>
|
#include <DB/Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
|
||||||
#include <DB/Interpreters/AggregationCommon.h>
|
#include <DB/Interpreters/AggregationCommon.h>
|
||||||
|
#include <DB/Interpreters/SettingsCommon.h>
|
||||||
|
|
||||||
#include <DB/Common/Arena.h>
|
#include <DB/Common/Arena.h>
|
||||||
#include <DB/Common/HashTable/HashMap.h>
|
#include <DB/Common/HashTable/HashMap.h>
|
||||||
@ -15,6 +16,8 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
struct Limits;
|
||||||
|
|
||||||
|
|
||||||
/** Структура данных для реализации JOIN-а.
|
/** Структура данных для реализации JOIN-а.
|
||||||
* По сути, хэш-таблица: ключи -> строки присоединяемой таблицы.
|
* По сути, хэш-таблица: ключи -> строки присоединяемой таблицы.
|
||||||
@ -62,16 +65,7 @@ class Join
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
Join(const Names & key_names_left_, const Names & key_names_right_,
|
Join(const Names & key_names_left_, const Names & key_names_right_,
|
||||||
const Limits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_)
|
const Limits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_);
|
||||||
: kind(kind_), strictness(strictness_),
|
|
||||||
key_names_left(key_names_left_),
|
|
||||||
key_names_right(key_names_right_),
|
|
||||||
log(&Logger::get("Join")),
|
|
||||||
max_rows(limits.max_rows_in_join),
|
|
||||||
max_bytes(limits.max_bytes_in_join),
|
|
||||||
overflow_mode(limits.join_overflow_mode)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
bool empty() { return type == Type::EMPTY; }
|
bool empty() { return type == Type::EMPTY; }
|
||||||
|
|
||||||
@ -229,7 +223,7 @@ private:
|
|||||||
Block sample_block_with_columns_to_add;
|
Block sample_block_with_columns_to_add;
|
||||||
Block sample_block_with_keys;
|
Block sample_block_with_keys;
|
||||||
|
|
||||||
Logger * log;
|
Poco::Logger * log;
|
||||||
|
|
||||||
/// Ограничения на максимальный размер множества
|
/// Ограничения на максимальный размер множества
|
||||||
size_t max_rows;
|
size_t max_rows;
|
||||||
|
@ -5,16 +5,12 @@
|
|||||||
#include <memory>
|
#include <memory>
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
#include <Poco/Condition.h>
|
#include <Poco/Condition.h>
|
||||||
#include <Poco/Net/IPAddress.h>
|
|
||||||
#include <DB/Common/Stopwatch.h>
|
#include <DB/Common/Stopwatch.h>
|
||||||
#include <DB/Core/Defines.h>
|
#include <DB/Core/Defines.h>
|
||||||
#include <DB/Core/Progress.h>
|
#include <DB/Core/Progress.h>
|
||||||
#include <DB/Common/Exception.h>
|
|
||||||
#include <DB/Common/MemoryTracker.h>
|
#include <DB/Common/MemoryTracker.h>
|
||||||
#include <DB/IO/WriteHelpers.h>
|
|
||||||
#include <DB/Interpreters/QueryPriorities.h>
|
#include <DB/Interpreters/QueryPriorities.h>
|
||||||
#include <DB/Interpreters/ClientInfo.h>
|
#include <DB/Interpreters/ClientInfo.h>
|
||||||
#include <DB/Storages/IStorage.h>
|
|
||||||
#include <DB/Common/CurrentMetrics.h>
|
#include <DB/Common/CurrentMetrics.h>
|
||||||
|
|
||||||
|
|
||||||
@ -26,6 +22,12 @@ namespace CurrentMetrics
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class IStorage;
|
||||||
|
using StoragePtr = std::shared_ptr<IStorage>;
|
||||||
|
using Tables = std::map<String, StoragePtr>;
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
/** List of currently executing queries.
|
/** List of currently executing queries.
|
||||||
* Also implements limit on their number.
|
* Also implements limit on their number.
|
||||||
*/
|
*/
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
|
|
||||||
#include <thread>
|
#include <thread>
|
||||||
#include <boost/noncopyable.hpp>
|
#include <boost/noncopyable.hpp>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
#include <DB/Common/ConcurrentBoundedQueue.h>
|
#include <DB/Common/ConcurrentBoundedQueue.h>
|
||||||
#include <DB/Storages/IStorage.h>
|
#include <DB/Storages/IStorage.h>
|
||||||
|
@ -6,10 +6,6 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
/// Для RESHARD PARTITION.
|
|
||||||
using WeightedZooKeeperPath = std::pair<String, UInt64>;
|
|
||||||
using WeightedZooKeeperPaths = std::vector<WeightedZooKeeperPath>;
|
|
||||||
|
|
||||||
/// Операция из запроса ALTER (кроме манипуляции с PART/PARTITION). Добавление столбцов типа Nested не развернуто в добавление отдельных столбцов.
|
/// Операция из запроса ALTER (кроме манипуляции с PART/PARTITION). Добавление столбцов типа Nested не развернуто в добавление отдельных столбцов.
|
||||||
struct AlterCommand
|
struct AlterCommand
|
||||||
{
|
{
|
||||||
|
@ -1,18 +1,9 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <common/logger_useful.h>
|
|
||||||
|
|
||||||
#include <DB/Core/Defines.h>
|
|
||||||
#include <DB/Core/Names.h>
|
#include <DB/Core/Names.h>
|
||||||
#include <DB/Core/NamesAndTypes.h>
|
|
||||||
#include <DB/Common/Exception.h>
|
#include <DB/Common/Exception.h>
|
||||||
#include <DB/Core/QueryProcessingStage.h>
|
#include <DB/Core/QueryProcessingStage.h>
|
||||||
#include <DB/Parsers/IAST.h>
|
|
||||||
#include <DB/Parsers/ASTAlterQuery.h>
|
|
||||||
#include <DB/Interpreters/Settings.h>
|
|
||||||
#include <DB/Storages/ITableDeclaration.h>
|
#include <DB/Storages/ITableDeclaration.h>
|
||||||
#include <DB/Storages/AlterCommands.h>
|
|
||||||
#include <Poco/File.h>
|
|
||||||
#include <Poco/RWLock.h>
|
#include <Poco/RWLock.h>
|
||||||
#include <memory>
|
#include <memory>
|
||||||
#include <experimental/optional>
|
#include <experimental/optional>
|
||||||
@ -39,6 +30,47 @@ class IStorage;
|
|||||||
|
|
||||||
using StoragePtr = std::shared_ptr<IStorage>;
|
using StoragePtr = std::shared_ptr<IStorage>;
|
||||||
|
|
||||||
|
class IAST;
|
||||||
|
|
||||||
|
using ASTPtr = std::shared_ptr<IAST>;
|
||||||
|
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
class AlterCommands;
|
||||||
|
|
||||||
|
/// For RESHARD PARTITION.
|
||||||
|
using WeightedZooKeeperPath = std::pair<String, UInt64>;
|
||||||
|
using WeightedZooKeeperPaths = std::vector<WeightedZooKeeperPath>;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/** Не дает изменять описание таблицы (в том числе переименовывать и удалять таблицу).
|
||||||
|
* Если в течение какой-то операции структура таблицы должна оставаться неизменной, нужно держать такой лок на все ее время.
|
||||||
|
* Например, нужно держать такой лок на время всего запроса SELECT или INSERT и на все время слияния набора кусков
|
||||||
|
* (но между выбором кусков для слияния и их слиянием структура таблицы может измениться).
|
||||||
|
* NOTE: Это лок на "чтение" описания таблицы. Чтобы изменить описание таблицы, нужно взять TableStructureWriteLock.
|
||||||
|
*/
|
||||||
|
class TableStructureReadLock
|
||||||
|
{
|
||||||
|
private:
|
||||||
|
friend class IStorage;
|
||||||
|
|
||||||
|
StoragePtr storage;
|
||||||
|
/// Порядок важен.
|
||||||
|
std::experimental::optional<Poco::ScopedReadRWLock> data_lock;
|
||||||
|
std::experimental::optional<Poco::ScopedReadRWLock> structure_lock;
|
||||||
|
|
||||||
|
public:
|
||||||
|
TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data);
|
||||||
|
};
|
||||||
|
|
||||||
|
using TableStructureReadLockPtr = std::shared_ptr<TableStructureReadLock>;
|
||||||
|
using TableStructureReadLocks = std::vector<TableStructureReadLockPtr>;
|
||||||
|
|
||||||
|
using TableStructureWriteLockPtr = std::unique_ptr<Poco::ScopedWriteRWLock>;
|
||||||
|
using TableDataWriteLockPtr = std::unique_ptr<Poco::ScopedWriteRWLock>;
|
||||||
|
using TableFullWriteLockPtr = std::pair<TableDataWriteLockPtr, TableStructureWriteLockPtr>;
|
||||||
|
|
||||||
|
|
||||||
/** Хранилище. Отвечает за:
|
/** Хранилище. Отвечает за:
|
||||||
* - хранение данных таблицы;
|
* - хранение данных таблицы;
|
||||||
@ -68,35 +100,6 @@ public:
|
|||||||
/** Возвращает true, если хранилище поддерживает несколько реплик. */
|
/** Возвращает true, если хранилище поддерживает несколько реплик. */
|
||||||
virtual bool supportsParallelReplicas() const { return false; }
|
virtual bool supportsParallelReplicas() const { return false; }
|
||||||
|
|
||||||
/** Не дает изменять описание таблицы (в том числе переименовывать и удалять таблицу).
|
|
||||||
* Если в течение какой-то операции структура таблицы должна оставаться неизменной, нужно держать такой лок на все ее время.
|
|
||||||
* Например, нужно держать такой лок на время всего запроса SELECT или INSERT и на все время слияния набора кусков
|
|
||||||
* (но между выбором кусков для слияния и их слиянием структура таблицы может измениться).
|
|
||||||
* NOTE: Это лок на "чтение" описания таблицы. Чтобы изменить описание таблицы, нужно взять TableStructureWriteLock.
|
|
||||||
*/
|
|
||||||
class TableStructureReadLock
|
|
||||||
{
|
|
||||||
private:
|
|
||||||
friend class IStorage;
|
|
||||||
|
|
||||||
StoragePtr storage;
|
|
||||||
/// Порядок важен.
|
|
||||||
std::experimental::optional<Poco::ScopedReadRWLock> data_lock;
|
|
||||||
std::experimental::optional<Poco::ScopedReadRWLock> structure_lock;
|
|
||||||
|
|
||||||
public:
|
|
||||||
TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data) : storage(storage_)
|
|
||||||
{
|
|
||||||
if (lock_data)
|
|
||||||
data_lock.emplace(storage->data_lock);
|
|
||||||
if (lock_structure)
|
|
||||||
structure_lock.emplace(storage->structure_lock);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
using TableStructureReadLockPtr = std::shared_ptr<TableStructureReadLock>;
|
|
||||||
using TableStructureReadLocks = std::vector<TableStructureReadLockPtr>;
|
|
||||||
|
|
||||||
/** Не дает изменять структуру или имя таблицы.
|
/** Не дает изменять структуру или имя таблицы.
|
||||||
* Если в рамках этого лока будут изменены данные в таблице, нужно указать will_modify_data=true.
|
* Если в рамках этого лока будут изменены данные в таблице, нужно указать will_modify_data=true.
|
||||||
* Это возьмет дополнительный лок, не позволяющий начать ALTER MODIFY.
|
* Это возьмет дополнительный лок, не позволяющий начать ALTER MODIFY.
|
||||||
@ -112,10 +115,6 @@ public:
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
using TableStructureWriteLockPtr = std::unique_ptr<Poco::ScopedWriteRWLock>;
|
|
||||||
using TableDataWriteLockPtr = std::unique_ptr<Poco::ScopedWriteRWLock>;
|
|
||||||
using TableFullWriteLockPtr = std::pair<TableDataWriteLockPtr, TableStructureWriteLockPtr>;
|
|
||||||
|
|
||||||
/** Не дает читать структуру таблицы. Берется для ALTER, RENAME и DROP.
|
/** Не дает читать структуру таблицы. Берется для ALTER, RENAME и DROP.
|
||||||
*/
|
*/
|
||||||
TableFullWriteLockPtr lockForAlter()
|
TableFullWriteLockPtr lockForAlter()
|
||||||
@ -283,6 +282,8 @@ protected:
|
|||||||
using std::enable_shared_from_this<IStorage>::shared_from_this;
|
using std::enable_shared_from_this<IStorage>::shared_from_this;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
friend class TableStructureReadLock;
|
||||||
|
|
||||||
/// Брать следующие два лока всегда нужно в этом порядке.
|
/// Брать следующие два лока всегда нужно в этом порядке.
|
||||||
|
|
||||||
/** Берется на чтение на все время запроса INSERT и на все время слияния кусков (для MergeTree).
|
/** Берется на чтение на все время запроса INSERT и на все время слияния кусков (для MergeTree).
|
||||||
@ -307,7 +308,6 @@ private:
|
|||||||
};
|
};
|
||||||
|
|
||||||
using StorageVector = std::vector<StoragePtr>;
|
using StorageVector = std::vector<StoragePtr>;
|
||||||
using TableLocks = IStorage::TableStructureReadLocks;
|
|
||||||
|
|
||||||
/// имя таблицы -> таблица
|
/// имя таблицы -> таблица
|
||||||
using Tables = std::map<String, StoragePtr>;
|
using Tables = std::map<String, StoragePtr>;
|
||||||
|
@ -1,11 +1,13 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Storages/AlterCommands.h>
|
#include <DB/Storages/AlterCommands.h>
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
#include <string>
|
#include <string>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
||||||
class StorageReplicatedMergeTree;
|
class StorageReplicatedMergeTree;
|
||||||
|
|
||||||
/** Описание задачи перешардирования.
|
/** Описание задачи перешардирования.
|
||||||
|
@ -10,6 +10,9 @@
|
|||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace Poco { class Logger; }
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -115,7 +118,7 @@ private:
|
|||||||
const String destination_table;
|
const String destination_table;
|
||||||
bool no_destination; /// Если задано - не записывать данные из буфера, а просто опустошать буфер.
|
bool no_destination; /// Если задано - не записывать данные из буфера, а просто опустошать буфер.
|
||||||
|
|
||||||
Logger * log;
|
Poco::Logger * log;
|
||||||
|
|
||||||
Poco::Event shutdown_event;
|
Poco::Event shutdown_event;
|
||||||
/// Выполняет сброс данных по таймауту.
|
/// Выполняет сброс данных по таймауту.
|
||||||
|
@ -1,12 +1,19 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DB/Storages/IStorage.h>
|
#include <string>
|
||||||
#include <DB/Parsers/ASTFunction.h>
|
#include <memory>
|
||||||
#include <DB/Interpreters/Context.h>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class Context;
|
||||||
|
class IAST;
|
||||||
|
using ASTPtr = std::shared_ptr<IAST>;
|
||||||
|
class IStorage;
|
||||||
|
using StoragePtr = std::shared_ptr<IStorage>;
|
||||||
|
|
||||||
|
|
||||||
/** Интерфейс для табличных функций.
|
/** Интерфейс для табличных функций.
|
||||||
*
|
*
|
||||||
* Табличные функции не имеют отношения к другим функциям.
|
* Табличные функции не имеют отношения к другим функциям.
|
||||||
|
@ -12,7 +12,7 @@ class TableFunctionFactory
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
TableFunctionPtr get(
|
TableFunctionPtr get(
|
||||||
const String & name,
|
const std::string & name,
|
||||||
const Context & context) const;
|
const Context & context) const;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <DB/Parsers/ASTSelectQuery.h>
|
#include <DB/Parsers/ASTSelectQuery.h>
|
||||||
#include <DB/Parsers/ASTTablesInSelectQuery.h>
|
#include <DB/Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
|
#include <DB/Parsers/ASTFunction.h>
|
||||||
#include <DB/Parsers/formatAST.h>
|
#include <DB/Parsers/formatAST.h>
|
||||||
#include <DB/Parsers/ASTSubquery.h>
|
#include <DB/Parsers/ASTSubquery.h>
|
||||||
#include <DB/IO/WriteBuffer.h>
|
#include <DB/IO/WriteBuffer.h>
|
||||||
|
@ -9,52 +9,6 @@
|
|||||||
#include <DB/Core/FieldVisitors.h>
|
#include <DB/Core/FieldVisitors.h>
|
||||||
|
|
||||||
|
|
||||||
/// This is for Yandex.Metrica code.
|
|
||||||
namespace mysqlxx
|
|
||||||
{
|
|
||||||
std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Array & value)
|
|
||||||
{
|
|
||||||
return res.ostr << DB::applyVisitor(DB::FieldVisitorToString(), DB::Field(value));
|
|
||||||
}
|
|
||||||
|
|
||||||
std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Array & value)
|
|
||||||
{
|
|
||||||
throw Poco::Exception("Cannot quote Array with mysqlxx::quote.");
|
|
||||||
}
|
|
||||||
|
|
||||||
std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Array & value)
|
|
||||||
{
|
|
||||||
throw Poco::Exception("Cannot unescape Array with mysqlxx::unescape.");
|
|
||||||
}
|
|
||||||
|
|
||||||
std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Array & value)
|
|
||||||
{
|
|
||||||
throw Poco::Exception("Cannot unquote Array with mysqlxx::unquote.");
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Tuple & value)
|
|
||||||
{
|
|
||||||
return res.ostr << DB::applyVisitor(DB::FieldVisitorToString(), DB::Field(value));
|
|
||||||
}
|
|
||||||
|
|
||||||
std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Tuple & value)
|
|
||||||
{
|
|
||||||
throw Poco::Exception("Cannot quote Tuple with mysqlxx::quote.");
|
|
||||||
}
|
|
||||||
|
|
||||||
std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Tuple & value)
|
|
||||||
{
|
|
||||||
throw Poco::Exception("Cannot unescape Tuple with mysqlxx::unescape.");
|
|
||||||
}
|
|
||||||
|
|
||||||
std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Tuple & value)
|
|
||||||
{
|
|
||||||
throw Poco::Exception("Cannot unquote Tuple with mysqlxx::unquote.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
inline void readBinary(Array & x, ReadBuffer & buf)
|
inline void readBinary(Array & x, ReadBuffer & buf)
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
#include <DB/Core/Block.h>
|
||||||
#include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h>
|
#include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,6 +1,8 @@
|
|||||||
#include <DB/DataStreams/ColumnGathererStream.h>
|
#include <DB/DataStreams/ColumnGathererStream.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
#include <iomanip>
|
#include <iomanip>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -15,7 +17,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
ColumnGathererStream::ColumnGathererStream(const BlockInputStreams & source_streams, const String & column_name_,
|
ColumnGathererStream::ColumnGathererStream(const BlockInputStreams & source_streams, const String & column_name_,
|
||||||
const MergedRowSources & row_source_, size_t block_preferred_size_)
|
const MergedRowSources & row_source_, size_t block_preferred_size_)
|
||||||
: name(column_name_), row_source(row_source_), block_preferred_size(block_preferred_size_)
|
: name(column_name_), row_source(row_source_), block_preferred_size(block_preferred_size_), log(&Logger::get("ColumnGathererStream"))
|
||||||
{
|
{
|
||||||
if (source_streams.empty())
|
if (source_streams.empty())
|
||||||
throw Exception("There are no streams to gather", ErrorCodes::EMPTY_DATA_PASSED);
|
throw Exception("There are no streams to gather", ErrorCodes::EMPTY_DATA_PASSED);
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <DB/Interpreters/Join.h>
|
#include <DB/Interpreters/Join.h>
|
||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
#include <DB/DataStreams/CreatingSetsBlockInputStream.h>
|
#include <DB/DataStreams/CreatingSetsBlockInputStream.h>
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
#include <iomanip>
|
#include <iomanip>
|
||||||
|
|
||||||
|
|
||||||
|
37
dbms/src/DataStreams/MaterializingBlockOutputStream.cpp
Normal file
37
dbms/src/DataStreams/MaterializingBlockOutputStream.cpp
Normal file
@ -0,0 +1,37 @@
|
|||||||
|
#include <DB/Columns/ColumnConst.h>
|
||||||
|
#include <DB/DataStreams/MaterializingBlockOutputStream.h>
|
||||||
|
#include <DB/DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
|
#include <ext/range.hpp>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
Block MaterializingBlockOutputStream::materialize(const Block & original_block)
|
||||||
|
{
|
||||||
|
/// copy block to get rid of const
|
||||||
|
auto block = original_block;
|
||||||
|
|
||||||
|
for (const auto i : ext::range(0, block.columns()))
|
||||||
|
{
|
||||||
|
auto & element = block.safeGetByPosition(i);
|
||||||
|
auto & src = element.column;
|
||||||
|
ColumnPtr converted = src->convertToFullColumnIfConst();
|
||||||
|
if (converted)
|
||||||
|
{
|
||||||
|
src = converted;
|
||||||
|
auto & type = element.type;
|
||||||
|
if (type->isNull())
|
||||||
|
{
|
||||||
|
/// A ColumnNull that is converted to a full column
|
||||||
|
/// has the type DataTypeNullable(DataTypeUInt8).
|
||||||
|
type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return block;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -1,4 +1,5 @@
|
|||||||
#include <DB/Core/Defines.h>
|
#include <DB/Core/Defines.h>
|
||||||
|
#include <DB/Core/Block.h>
|
||||||
|
|
||||||
#include <DB/IO/WriteHelpers.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
#include <DB/IO/VarInt.h>
|
#include <DB/IO/VarInt.h>
|
||||||
|
@ -1,5 +1,9 @@
|
|||||||
|
#include <DB/IO/WriteBuffer.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
#include <DB/Core/Block.h>
|
||||||
#include <DB/DataStreams/ODBCDriverBlockOutputStream.h>
|
#include <DB/DataStreams/ODBCDriverBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -16,15 +20,15 @@ void ODBCDriverBlockOutputStream::write(const Block & block)
|
|||||||
size_t rows = block.rows();
|
size_t rows = block.rows();
|
||||||
size_t columns = block.columns();
|
size_t columns = block.columns();
|
||||||
|
|
||||||
/// Заголовок.
|
/// Header.
|
||||||
if (is_first)
|
if (is_first)
|
||||||
{
|
{
|
||||||
is_first = false;
|
is_first = false;
|
||||||
|
|
||||||
/// Количество столбцов.
|
/// Number of columns.
|
||||||
writeVarUInt(columns, out);
|
writeVarUInt(columns, out);
|
||||||
|
|
||||||
/// Имена и типы столбцов.
|
/// Names and types of columns.
|
||||||
for (size_t j = 0; j < columns; ++j)
|
for (size_t j = 0; j < columns; ++j)
|
||||||
{
|
{
|
||||||
const ColumnWithTypeAndName & col = block.getByPosition(j);
|
const ColumnWithTypeAndName & col = block.getByPosition(j);
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
#include <sys/ioctl.h>
|
#include <DB/IO/WriteBuffer.h>
|
||||||
#include <unistd.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/PrettyCompactBlockOutputStream.h>
|
#include <DB/DataStreams/PrettyCompactBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
#include <sys/ioctl.h>
|
#include <DB/IO/WriteBuffer.h>
|
||||||
#include <unistd.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/PrettyCompactMonoBlockOutputStream.h>
|
#include <DB/DataStreams/PrettyCompactMonoBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
#include <sys/ioctl.h>
|
#include <DB/IO/WriteBuffer.h>
|
||||||
#include <unistd.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/PrettySpaceBlockOutputStream.h>
|
#include <DB/DataStreams/PrettySpaceBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
17
dbms/src/DataStreams/ProhibitColumnsBlockOutputStream.cpp
Normal file
17
dbms/src/DataStreams/ProhibitColumnsBlockOutputStream.cpp
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
#include <DB/Core/Block.h>
|
||||||
|
#include <DB/DataStreams/ProhibitColumnsBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void ProhibitColumnsBlockOutputStream::write(const Block & block)
|
||||||
|
{
|
||||||
|
for (const auto & column : columns)
|
||||||
|
if (block.has(column.name))
|
||||||
|
throw Exception{"Cannot insert column " + column.name, ErrorCodes::ILLEGAL_COLUMN};
|
||||||
|
|
||||||
|
output->write(block);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -2,6 +2,9 @@
|
|||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
#include <DB/Common/VirtualColumnUtils.h>
|
#include <DB/Common/VirtualColumnUtils.h>
|
||||||
#include <DB/Common/NetException.h>
|
#include <DB/Common/NetException.h>
|
||||||
|
#include <DB/Interpreters/Context.h>
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
#include <DB/IO/WriteBuffer.h>
|
||||||
|
#include <DB/Core/Block.h>
|
||||||
#include <DB/DataStreams/TabSeparatedBlockOutputStream.h>
|
#include <DB/DataStreams/TabSeparatedBlockOutputStream.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,6 +1,8 @@
|
|||||||
|
#include <common/logger_useful.h>
|
||||||
#include <DB/Databases/DatabaseMemory.h>
|
#include <DB/Databases/DatabaseMemory.h>
|
||||||
#include <DB/Databases/DatabasesCommon.h>
|
#include <DB/Databases/DatabasesCommon.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
#include <future>
|
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Databases/DatabaseOrdinary.h>
|
#include <DB/Databases/DatabaseOrdinary.h>
|
||||||
#include <DB/Databases/DatabaseMemory.h>
|
#include <DB/Databases/DatabaseMemory.h>
|
||||||
|
52
dbms/src/Dictionaries/Embedded/RegionsHierarchies.cpp
Normal file
52
dbms/src/Dictionaries/Embedded/RegionsHierarchies.cpp
Normal file
@ -0,0 +1,52 @@
|
|||||||
|
#include <DB/Dictionaries/Embedded/RegionsHierarchies.h>
|
||||||
|
#include <Poco/DirectoryIterator.h>
|
||||||
|
#include <Poco/Util/Application.h>
|
||||||
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
|
static constexpr auto config_key = "path_to_regions_hierarchy_file";
|
||||||
|
|
||||||
|
|
||||||
|
RegionsHierarchies::RegionsHierarchies()
|
||||||
|
{
|
||||||
|
Logger * log = &Logger::get("RegionsHierarchies");
|
||||||
|
|
||||||
|
std::string path = Poco::Util::Application::instance().config().getString(config_key);
|
||||||
|
|
||||||
|
LOG_DEBUG(log, "Adding default regions hierarchy from " << path);
|
||||||
|
|
||||||
|
data.emplace(std::piecewise_construct,
|
||||||
|
std::forward_as_tuple(""),
|
||||||
|
std::forward_as_tuple(path));
|
||||||
|
|
||||||
|
std::string basename = Poco::Path(path).getBaseName();
|
||||||
|
|
||||||
|
Poco::Path dir_path = Poco::Path(path).absolute().parent();
|
||||||
|
|
||||||
|
Poco::DirectoryIterator dir_end;
|
||||||
|
for (Poco::DirectoryIterator dir_it(dir_path); dir_it != dir_end; ++dir_it)
|
||||||
|
{
|
||||||
|
std::string other_basename = dir_it.path().getBaseName();
|
||||||
|
|
||||||
|
if (0 == other_basename.compare(0, basename.size(), basename) && other_basename.size() > basename.size() + 1)
|
||||||
|
{
|
||||||
|
if (other_basename[basename.size()] != '_')
|
||||||
|
continue;
|
||||||
|
|
||||||
|
std::string suffix = other_basename.substr(basename.size() + 1);
|
||||||
|
|
||||||
|
LOG_DEBUG(log, "Adding regions hierarchy from " << dir_it->path() << ", key: " << suffix);
|
||||||
|
|
||||||
|
data.emplace(std::piecewise_construct,
|
||||||
|
std::forward_as_tuple(suffix),
|
||||||
|
std::forward_as_tuple(dir_it->path()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool RegionsHierarchies::isConfigured()
|
||||||
|
{
|
||||||
|
return Poco::Util::Application::instance().config().has(config_key);
|
||||||
|
}
|
198
dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp
Normal file
198
dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp
Normal file
@ -0,0 +1,198 @@
|
|||||||
|
#include <DB/Dictionaries/Embedded/RegionsHierarchy.h>
|
||||||
|
|
||||||
|
#include <Poco/Util/Application.h>
|
||||||
|
#include <Poco/Exception.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include <common/singleton.h>
|
||||||
|
|
||||||
|
#include <DB/IO/ReadBufferFromFile.h>
|
||||||
|
#include <DB/IO/ReadHelpers.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
|
static constexpr auto config_key = "path_to_regions_hierarchy_file";
|
||||||
|
|
||||||
|
|
||||||
|
RegionsHierarchy::RegionsHierarchy()
|
||||||
|
{
|
||||||
|
path = Poco::Util::Application::instance().config().getString(config_key);
|
||||||
|
}
|
||||||
|
|
||||||
|
RegionsHierarchy::RegionsHierarchy(const std::string & path_)
|
||||||
|
{
|
||||||
|
path = path_;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void RegionsHierarchy::reload()
|
||||||
|
{
|
||||||
|
Logger * log = &Logger::get("RegionsHierarchy");
|
||||||
|
|
||||||
|
time_t new_modification_time = Poco::File(path).getLastModified().epochTime();
|
||||||
|
if (new_modification_time <= file_modification_time)
|
||||||
|
return;
|
||||||
|
file_modification_time = new_modification_time;
|
||||||
|
|
||||||
|
LOG_DEBUG(log, "Reloading regions hierarchy");
|
||||||
|
|
||||||
|
const size_t initial_size = 10000;
|
||||||
|
const size_t max_size = 1000000;
|
||||||
|
|
||||||
|
RegionParents new_parents(initial_size);
|
||||||
|
RegionParents new_city(initial_size);
|
||||||
|
RegionParents new_country(initial_size);
|
||||||
|
RegionParents new_area(initial_size);
|
||||||
|
RegionParents new_district(initial_size);
|
||||||
|
RegionParents new_continent(initial_size);
|
||||||
|
RegionParents new_top_continent(initial_size);
|
||||||
|
RegionPopulations new_populations(initial_size);
|
||||||
|
RegionDepths new_depths(initial_size);
|
||||||
|
RegionTypes types(initial_size);
|
||||||
|
|
||||||
|
DB::ReadBufferFromFile in(path);
|
||||||
|
|
||||||
|
RegionID max_region_id = 0;
|
||||||
|
while (!in.eof())
|
||||||
|
{
|
||||||
|
/** Our internal geobase has negative numbers,
|
||||||
|
* that means "this is garbage, ignore this row".
|
||||||
|
*/
|
||||||
|
Int32 read_region_id = 0;
|
||||||
|
Int32 read_parent_id = 0;
|
||||||
|
Int8 read_type = 0;
|
||||||
|
|
||||||
|
DB::readIntText(read_region_id, in);
|
||||||
|
DB::assertChar('\t', in);
|
||||||
|
DB::readIntText(read_parent_id, in);
|
||||||
|
DB::assertChar('\t', in);
|
||||||
|
DB::readIntText(read_type, in);
|
||||||
|
|
||||||
|
/** Далее может быть перевод строки (старый вариант)
|
||||||
|
* или таб, население региона, перевод строки (новый вариант).
|
||||||
|
*/
|
||||||
|
RegionPopulation population = 0;
|
||||||
|
if (!in.eof() && *in.position() == '\t')
|
||||||
|
{
|
||||||
|
++in.position();
|
||||||
|
UInt64 population_big = 0;
|
||||||
|
DB::readIntText(population_big, in);
|
||||||
|
population = population_big > std::numeric_limits<RegionPopulation>::max()
|
||||||
|
? std::numeric_limits<RegionPopulation>::max()
|
||||||
|
: population_big;
|
||||||
|
}
|
||||||
|
DB::assertChar('\n', in);
|
||||||
|
|
||||||
|
if (read_region_id <= 0 || read_type < 0)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
RegionID region_id = read_region_id;
|
||||||
|
RegionID parent_id = 0;
|
||||||
|
|
||||||
|
if (read_parent_id >= 0)
|
||||||
|
parent_id = read_parent_id;
|
||||||
|
|
||||||
|
RegionType type = read_type;
|
||||||
|
|
||||||
|
if (region_id > max_region_id)
|
||||||
|
{
|
||||||
|
if (region_id > max_size)
|
||||||
|
throw DB::Exception("Region id is too large: " + DB::toString(region_id) + ", should be not more than " + DB::toString(max_size));
|
||||||
|
|
||||||
|
max_region_id = region_id;
|
||||||
|
|
||||||
|
while (region_id >= new_parents.size())
|
||||||
|
{
|
||||||
|
new_parents.resize(new_parents.size() * 2);
|
||||||
|
new_populations.resize(new_parents.size());
|
||||||
|
types.resize(new_parents.size());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
new_parents[region_id] = parent_id;
|
||||||
|
new_populations[region_id] = population;
|
||||||
|
types[region_id] = type;
|
||||||
|
}
|
||||||
|
|
||||||
|
new_parents .resize(max_region_id + 1);
|
||||||
|
new_city .resize(max_region_id + 1);
|
||||||
|
new_country .resize(max_region_id + 1);
|
||||||
|
new_area .resize(max_region_id + 1);
|
||||||
|
new_district .resize(max_region_id + 1);
|
||||||
|
new_continent .resize(max_region_id + 1);
|
||||||
|
new_top_continent.resize(max_region_id + 1);
|
||||||
|
new_populations .resize(max_region_id + 1);
|
||||||
|
new_depths .resize(max_region_id + 1);
|
||||||
|
types .resize(max_region_id + 1);
|
||||||
|
|
||||||
|
/// пропишем города и страны для регионов
|
||||||
|
for (RegionID i = 0; i <= max_region_id; ++i)
|
||||||
|
{
|
||||||
|
if (types[i] == REGION_TYPE_CITY)
|
||||||
|
new_city[i] = i;
|
||||||
|
|
||||||
|
if (types[i] == REGION_TYPE_AREA)
|
||||||
|
new_area[i] = i;
|
||||||
|
|
||||||
|
if (types[i] == REGION_TYPE_DISTRICT)
|
||||||
|
new_district[i] = i;
|
||||||
|
|
||||||
|
if (types[i] == REGION_TYPE_COUNTRY)
|
||||||
|
new_country[i] = i;
|
||||||
|
|
||||||
|
if (types[i] == REGION_TYPE_CONTINENT)
|
||||||
|
{
|
||||||
|
new_continent[i] = i;
|
||||||
|
new_top_continent[i] = i;
|
||||||
|
}
|
||||||
|
|
||||||
|
RegionDepth depth = 0;
|
||||||
|
RegionID current = i;
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
++depth;
|
||||||
|
|
||||||
|
if (depth == std::numeric_limits<RegionDepth>::max())
|
||||||
|
throw Poco::Exception("Logical error in regions hierarchy: region " + DB::toString(current) + " possible is inside infinite loop");
|
||||||
|
|
||||||
|
current = new_parents[current];
|
||||||
|
if (current == 0)
|
||||||
|
break;
|
||||||
|
|
||||||
|
if (current > max_region_id)
|
||||||
|
throw Poco::Exception("Logical error in regions hierarchy: region " + DB::toString(current) + " (specified as parent) doesn't exist");
|
||||||
|
|
||||||
|
if (types[current] == REGION_TYPE_CITY)
|
||||||
|
new_city[i] = current;
|
||||||
|
|
||||||
|
if (types[current] == REGION_TYPE_AREA)
|
||||||
|
new_area[i] = current;
|
||||||
|
|
||||||
|
if (types[current] == REGION_TYPE_DISTRICT)
|
||||||
|
new_district[i] = current;
|
||||||
|
|
||||||
|
if (types[current] == REGION_TYPE_COUNTRY)
|
||||||
|
new_country[i] = current;
|
||||||
|
|
||||||
|
if (types[current] == REGION_TYPE_CONTINENT)
|
||||||
|
{
|
||||||
|
if (!new_continent[i])
|
||||||
|
new_continent[i] = current;
|
||||||
|
new_top_continent[i] = current;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
new_depths[i] = depth;
|
||||||
|
}
|
||||||
|
|
||||||
|
parents.swap(new_parents);
|
||||||
|
country.swap(new_country);
|
||||||
|
city.swap(new_city);
|
||||||
|
area.swap(new_area);
|
||||||
|
district.swap(new_district);
|
||||||
|
continent.swap(new_continent);
|
||||||
|
top_continent.swap(new_top_continent);
|
||||||
|
populations.swap(new_populations);
|
||||||
|
depths.swap(new_depths);
|
||||||
|
}
|
113
dbms/src/Dictionaries/Embedded/RegionsNames.cpp
Normal file
113
dbms/src/Dictionaries/Embedded/RegionsNames.cpp
Normal file
@ -0,0 +1,113 @@
|
|||||||
|
#include <DB/Dictionaries/Embedded/RegionsNames.h>
|
||||||
|
|
||||||
|
#include <Poco/File.h>
|
||||||
|
#include <Poco/Util/Application.h>
|
||||||
|
#include <Poco/Exception.h>
|
||||||
|
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
#include <DB/IO/ReadHelpers.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
#include <DB/IO/ReadBufferFromFile.h>
|
||||||
|
|
||||||
|
|
||||||
|
static constexpr auto config_key = "path_to_regions_names_files";
|
||||||
|
|
||||||
|
|
||||||
|
std::string RegionsNames::dumpSupportedLanguagesNames()
|
||||||
|
{
|
||||||
|
std::string res = "";
|
||||||
|
for (size_t i = 0; i < LANGUAGE_ALIASES_COUNT; ++i)
|
||||||
|
{
|
||||||
|
if (i > 0)
|
||||||
|
res += ", ";
|
||||||
|
res += '\'';
|
||||||
|
res += getLanguageAliases()[i].name;
|
||||||
|
res += '\'';
|
||||||
|
}
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void RegionsNames::reload()
|
||||||
|
{
|
||||||
|
Logger * log = &Logger::get("RegionsNames");
|
||||||
|
LOG_DEBUG(log, "Reloading regions names");
|
||||||
|
|
||||||
|
std::string directory = Poco::Util::Application::instance().config().getString(config_key);
|
||||||
|
|
||||||
|
RegionID max_region_id = 0;
|
||||||
|
for (size_t language_id = 0; language_id < SUPPORTED_LANGUAGES_COUNT; ++language_id)
|
||||||
|
{
|
||||||
|
const std::string & language = getSupportedLanguages()[language_id];
|
||||||
|
std::string path = directory + "/regions_names_" + language + ".txt";
|
||||||
|
|
||||||
|
Poco::File file(path);
|
||||||
|
time_t new_modification_time = file.getLastModified().epochTime();
|
||||||
|
if (new_modification_time <= file_modification_times[language_id])
|
||||||
|
continue;
|
||||||
|
file_modification_times[language_id] = new_modification_time;
|
||||||
|
|
||||||
|
LOG_DEBUG(log, "Reloading regions names for language: " << language);
|
||||||
|
|
||||||
|
DB::ReadBufferFromFile in(path);
|
||||||
|
|
||||||
|
const size_t initial_size = 10000;
|
||||||
|
const size_t max_size = 1000000;
|
||||||
|
|
||||||
|
Chars new_chars;
|
||||||
|
StringRefs new_names_refs(initial_size, StringRef("", 0));
|
||||||
|
|
||||||
|
/// Выделим непрерывный кусок памяти, которого хватит для хранения всех имён.
|
||||||
|
new_chars.reserve(Poco::File(path).getSize());
|
||||||
|
|
||||||
|
while (!in.eof())
|
||||||
|
{
|
||||||
|
Int32 read_region_id;
|
||||||
|
std::string region_name;
|
||||||
|
|
||||||
|
DB::readIntText(read_region_id, in);
|
||||||
|
DB::assertChar('\t', in);
|
||||||
|
DB::readString(region_name, in);
|
||||||
|
DB::assertChar('\n', in);
|
||||||
|
|
||||||
|
if (read_region_id <= 0)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
RegionID region_id = read_region_id;
|
||||||
|
|
||||||
|
size_t old_size = new_chars.size();
|
||||||
|
|
||||||
|
if (new_chars.capacity() < old_size + region_name.length() + 1)
|
||||||
|
throw Poco::Exception("Logical error. Maybe size of file " + path + " is wrong.");
|
||||||
|
|
||||||
|
new_chars.resize(old_size + region_name.length() + 1);
|
||||||
|
memcpy(&new_chars[old_size], region_name.c_str(), region_name.length() + 1);
|
||||||
|
|
||||||
|
if (region_id > max_region_id)
|
||||||
|
{
|
||||||
|
max_region_id = region_id;
|
||||||
|
|
||||||
|
if (region_id > max_size)
|
||||||
|
throw DB::Exception("Region id is too large: " + DB::toString(region_id) + ", should be not more than " + DB::toString(max_size));
|
||||||
|
}
|
||||||
|
|
||||||
|
while (region_id >= new_names_refs.size())
|
||||||
|
new_names_refs.resize(new_names_refs.size() * 2, StringRef("", 0));
|
||||||
|
|
||||||
|
new_names_refs[region_id] = StringRef(&new_chars[old_size], region_name.length());
|
||||||
|
}
|
||||||
|
|
||||||
|
chars[language_id].swap(new_chars);
|
||||||
|
names_refs[language_id].swap(new_names_refs);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (size_t language_id = 0; language_id < SUPPORTED_LANGUAGES_COUNT; ++language_id)
|
||||||
|
names_refs[language_id].resize(max_region_id + 1, StringRef("", 0));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool RegionsNames::isConfigured()
|
||||||
|
{
|
||||||
|
return Poco::Util::Application::instance().config().has(config_key);
|
||||||
|
}
|
56
dbms/src/Dictionaries/Embedded/TechDataHierarchy.cpp
Normal file
56
dbms/src/Dictionaries/Embedded/TechDataHierarchy.cpp
Normal file
@ -0,0 +1,56 @@
|
|||||||
|
#include <DB/Dictionaries/Embedded/TechDataHierarchy.h>
|
||||||
|
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include <mysqlxx/PoolWithFailover.h>
|
||||||
|
|
||||||
|
|
||||||
|
static constexpr auto config_key = "mysql_metrica";
|
||||||
|
|
||||||
|
|
||||||
|
void TechDataHierarchy::reload()
|
||||||
|
{
|
||||||
|
Logger * log = &Logger::get("TechDataHierarchy");
|
||||||
|
|
||||||
|
LOG_DEBUG(log, "Loading tech data hierarchy.");
|
||||||
|
|
||||||
|
mysqlxx::PoolWithFailover pool(config_key);
|
||||||
|
mysqlxx::Pool::Entry conn = pool.Get();
|
||||||
|
|
||||||
|
{
|
||||||
|
mysqlxx::Query q = conn->query("SELECT Id, COALESCE(Parent_Id, 0) FROM OS2");
|
||||||
|
LOG_TRACE(log, q.str());
|
||||||
|
mysqlxx::UseQueryResult res = q.use();
|
||||||
|
while (mysqlxx::Row row = res.fetch())
|
||||||
|
{
|
||||||
|
UInt64 child = row[0].getUInt();
|
||||||
|
UInt64 parent = row[1].getUInt();
|
||||||
|
|
||||||
|
if (child > 255 || parent > 255)
|
||||||
|
throw Poco::Exception("Too large OS id (> 255).");
|
||||||
|
|
||||||
|
os_parent[child] = parent;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
mysqlxx::Query q = conn->query("SELECT Id, COALESCE(ParentId, 0) FROM SearchEngines");
|
||||||
|
LOG_TRACE(log, q.str());
|
||||||
|
mysqlxx::UseQueryResult res = q.use();
|
||||||
|
while (mysqlxx::Row row = res.fetch())
|
||||||
|
{
|
||||||
|
UInt64 child = row[0].getUInt();
|
||||||
|
UInt64 parent = row[1].getUInt();
|
||||||
|
|
||||||
|
if (child > 255 || parent > 255)
|
||||||
|
throw Poco::Exception("Too large search engine id (> 255).");
|
||||||
|
|
||||||
|
se_parent[child] = parent;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool TechDataHierarchy::isConfigured()
|
||||||
|
{
|
||||||
|
return Poco::Util::Application::instance().config().has(config_key);
|
||||||
|
}
|
@ -7,6 +7,9 @@
|
|||||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
|
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -15,7 +18,8 @@ static const size_t max_block_size = 8192;
|
|||||||
|
|
||||||
ExecutableDictionarySource::ExecutableDictionarySource(const DictionaryStructure & dict_struct_,
|
ExecutableDictionarySource::ExecutableDictionarySource(const DictionaryStructure & dict_struct_,
|
||||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||||
Block & sample_block, const Context & context) :
|
Block & sample_block, const Context & context)
|
||||||
|
: log(&Logger::get("ExecutableDictionarySource")),
|
||||||
dict_struct{dict_struct_},
|
dict_struct{dict_struct_},
|
||||||
command{config.getString(config_prefix + ".command")},
|
command{config.getString(config_prefix + ".command")},
|
||||||
format{config.getString(config_prefix + ".format")},
|
format{config.getString(config_prefix + ".format")},
|
||||||
@ -24,12 +28,13 @@ ExecutableDictionarySource::ExecutableDictionarySource(const DictionaryStructure
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other) :
|
ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other)
|
||||||
dict_struct{other.dict_struct},
|
: log(&Logger::get("ExecutableDictionarySource")),
|
||||||
command{other.command},
|
dict_struct{other.dict_struct},
|
||||||
format{other.format},
|
command{other.command},
|
||||||
sample_block{other.sample_block},
|
format{other.format},
|
||||||
context(other.context)
|
sample_block{other.sample_block},
|
||||||
|
context(other.context)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -93,12 +98,6 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64
|
|||||||
idsToBuffer(context, format, sample_block, process->in, ids);
|
idsToBuffer(context, format, sample_block, process->in, ids);
|
||||||
process->in.close();
|
process->in.close();
|
||||||
|
|
||||||
/*
|
|
||||||
std::string process_err;
|
|
||||||
readStringUntilEOF(process_err, process->err);
|
|
||||||
std::cerr << "readed STDERR [" << process_err << "] " << std::endl;
|
|
||||||
*/
|
|
||||||
|
|
||||||
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
||||||
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
|
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
|
||||||
}
|
}
|
||||||
@ -112,12 +111,6 @@ BlockInputStreamPtr ExecutableDictionarySource::loadKeys(
|
|||||||
columnsToBuffer(context, format, sample_block, process->in, dict_struct, key_columns, requested_rows);
|
columnsToBuffer(context, format, sample_block, process->in, dict_struct, key_columns, requested_rows);
|
||||||
process->in.close();
|
process->in.close();
|
||||||
|
|
||||||
/*
|
|
||||||
std::string process_err;
|
|
||||||
readStringUntilEOF(process_err, process->err);
|
|
||||||
std::cerr << "readed STDERR [" << process_err << "] " << std::endl;
|
|
||||||
*/
|
|
||||||
|
|
||||||
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
auto stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
||||||
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
|
return std::make_shared<OwningBlockInputStream<ShellCommand>>(stream, std::move(process));
|
||||||
}
|
}
|
||||||
|
@ -12,6 +12,9 @@
|
|||||||
|
|
||||||
#include <DB/Dictionaries/ExecutableDictionarySource.h> // idsToBuffer, columnsToBuffer
|
#include <DB/Dictionaries/ExecutableDictionarySource.h> // idsToBuffer, columnsToBuffer
|
||||||
|
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -19,8 +22,9 @@ static const size_t max_block_size = 8192;
|
|||||||
|
|
||||||
|
|
||||||
HTTPDictionarySource::HTTPDictionarySource(const DictionaryStructure & dict_struct_,
|
HTTPDictionarySource::HTTPDictionarySource(const DictionaryStructure & dict_struct_,
|
||||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||||
Block & sample_block, const Context & context) :
|
Block & sample_block, const Context & context)
|
||||||
|
: log(&Logger::get("HTTPDictionarySource")),
|
||||||
dict_struct{dict_struct_},
|
dict_struct{dict_struct_},
|
||||||
url{config.getString(config_prefix + ".url", "")},
|
url{config.getString(config_prefix + ".url", "")},
|
||||||
format{config.getString(config_prefix + ".format")},
|
format{config.getString(config_prefix + ".format")},
|
||||||
@ -29,7 +33,8 @@ HTTPDictionarySource::HTTPDictionarySource(const DictionaryStructure & dict_stru
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) :
|
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
|
||||||
|
: log(&Logger::get("HTTPDictionarySource")),
|
||||||
dict_struct{other.dict_struct},
|
dict_struct{other.dict_struct},
|
||||||
url{other.url},
|
url{other.url},
|
||||||
sample_block{other.sample_block},
|
sample_block{other.sample_block},
|
||||||
|
@ -1,4 +1,7 @@
|
|||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Dictionaries/MySQLDictionarySource.h>
|
#include <DB/Dictionaries/MySQLDictionarySource.h>
|
||||||
|
#include <DB/Dictionaries/MySQLBlockInputStream.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -11,29 +14,31 @@ static const size_t max_block_size = 8192;
|
|||||||
MySQLDictionarySource::MySQLDictionarySource(const DictionaryStructure & dict_struct_,
|
MySQLDictionarySource::MySQLDictionarySource(const DictionaryStructure & dict_struct_,
|
||||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||||
const Block & sample_block)
|
const Block & sample_block)
|
||||||
: dict_struct{dict_struct_},
|
: log(&Logger::get("MySQLDictionarySource")),
|
||||||
db{config.getString(config_prefix + ".db", "")},
|
dict_struct{dict_struct_},
|
||||||
table{config.getString(config_prefix + ".table")},
|
db{config.getString(config_prefix + ".db", "")},
|
||||||
where{config.getString(config_prefix + ".where", "")},
|
table{config.getString(config_prefix + ".table")},
|
||||||
dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)},
|
where{config.getString(config_prefix + ".where", "")},
|
||||||
sample_block{sample_block},
|
dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)},
|
||||||
pool{config, config_prefix},
|
sample_block{sample_block},
|
||||||
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
|
pool{config, config_prefix},
|
||||||
load_all_query{query_builder.composeLoadAllQuery()}
|
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
|
||||||
|
load_all_query{query_builder.composeLoadAllQuery()}
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
/// copy-constructor is provided in order to support cloneability
|
/// copy-constructor is provided in order to support cloneability
|
||||||
MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other)
|
MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other)
|
||||||
: dict_struct{other.dict_struct},
|
: log(&Logger::get("MySQLDictionarySource")),
|
||||||
db{other.db},
|
dict_struct{other.dict_struct},
|
||||||
table{other.table},
|
db{other.db},
|
||||||
where{other.where},
|
table{other.table},
|
||||||
dont_check_update_time{other.dont_check_update_time},
|
where{other.where},
|
||||||
sample_block{other.sample_block},
|
dont_check_update_time{other.dont_check_update_time},
|
||||||
pool{other.pool},
|
sample_block{other.sample_block},
|
||||||
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
|
pool{other.pool},
|
||||||
load_all_query{other.load_all_query}, last_modification{other.last_modification}
|
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
|
||||||
|
load_all_query{other.load_all_query}, last_modification{other.last_modification}
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
#include <DB/Dictionaries/ODBCDictionarySource.h>
|
#include <DB/Dictionaries/ODBCDictionarySource.h>
|
||||||
#include <DB/Dictionaries/ODBCBlockInputStream.h>
|
#include <DB/Dictionaries/ODBCBlockInputStream.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -14,29 +15,31 @@ static const size_t max_block_size = 8192;
|
|||||||
ODBCDictionarySource::ODBCDictionarySource(const DictionaryStructure & dict_struct_,
|
ODBCDictionarySource::ODBCDictionarySource(const DictionaryStructure & dict_struct_,
|
||||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||||
const Block & sample_block)
|
const Block & sample_block)
|
||||||
: dict_struct{dict_struct_},
|
: log(&Logger::get("ODBCDictionarySource")),
|
||||||
db{config.getString(config_prefix + ".db", "")},
|
dict_struct{dict_struct_},
|
||||||
table{config.getString(config_prefix + ".table")},
|
db{config.getString(config_prefix + ".db", "")},
|
||||||
where{config.getString(config_prefix + ".where", "")},
|
table{config.getString(config_prefix + ".table")},
|
||||||
sample_block{sample_block},
|
where{config.getString(config_prefix + ".where", "")},
|
||||||
pool{std::make_shared<Poco::Data::SessionPool>(
|
sample_block{sample_block},
|
||||||
config.getString(config_prefix + ".connector", "ODBC"),
|
pool{std::make_shared<Poco::Data::SessionPool>(
|
||||||
config.getString(config_prefix + ".connection_string"))},
|
config.getString(config_prefix + ".connector", "ODBC"),
|
||||||
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::None}, /// NOTE Better to obtain quoting style via ODBC interface.
|
config.getString(config_prefix + ".connection_string"))},
|
||||||
load_all_query{query_builder.composeLoadAllQuery()}
|
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::None}, /// NOTE Better to obtain quoting style via ODBC interface.
|
||||||
|
load_all_query{query_builder.composeLoadAllQuery()}
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
/// copy-constructor is provided in order to support cloneability
|
/// copy-constructor is provided in order to support cloneability
|
||||||
ODBCDictionarySource::ODBCDictionarySource(const ODBCDictionarySource & other)
|
ODBCDictionarySource::ODBCDictionarySource(const ODBCDictionarySource & other)
|
||||||
: dict_struct{other.dict_struct},
|
: log(&Logger::get("ODBCDictionarySource")),
|
||||||
db{other.db},
|
dict_struct{other.dict_struct},
|
||||||
table{other.table},
|
db{other.db},
|
||||||
where{other.where},
|
table{other.table},
|
||||||
sample_block{other.sample_block},
|
where{other.where},
|
||||||
pool{other.pool},
|
sample_block{other.sample_block},
|
||||||
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::None},
|
pool{other.pool},
|
||||||
load_all_query{other.load_all_query}
|
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::None},
|
||||||
|
load_all_query{other.load_all_query}
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -8,6 +8,7 @@ namespace ErrorCodes
|
|||||||
{
|
{
|
||||||
extern const int BAD_ARGUMENTS;
|
extern const int BAD_ARGUMENTS;
|
||||||
extern const int DICTIONARY_IS_EMPTY;
|
extern const int DICTIONARY_IS_EMPTY;
|
||||||
|
extern const int TYPE_MISMATCH;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,3 +1,6 @@
|
|||||||
|
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||||
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
|
||||||
#include <DB/Functions/FunctionFactory.h>
|
#include <DB/Functions/FunctionFactory.h>
|
||||||
#include <DB/Functions/FunctionsArray.h>
|
#include <DB/Functions/FunctionsArray.h>
|
||||||
|
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <DB/Functions/DataTypeTraits.h>
|
#include <DB/Functions/DataTypeTraits.h>
|
||||||
#include <DB/DataTypes/DataTypeEnum.h>
|
#include <DB/DataTypes/DataTypeEnum.h>
|
||||||
#include <DB/DataTypes/DataTypeNullable.h>
|
#include <DB/DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
#include <DB/Columns/ColumnNullable.h>
|
#include <DB/Columns/ColumnNullable.h>
|
||||||
#include <common/ClickHouseRevision.h>
|
#include <common/ClickHouseRevision.h>
|
||||||
#include <ext/enumerate.hpp>
|
#include <ext/enumerate.hpp>
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <Poco/Mutex.h>
|
#include <Poco/Mutex.h>
|
||||||
#include <Poco/File.h>
|
#include <Poco/File.h>
|
||||||
#include <Poco/UUIDGenerator.h>
|
#include <Poco/UUIDGenerator.h>
|
||||||
|
#include <Poco/Net/IPAddress.h>
|
||||||
|
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
@ -24,7 +25,7 @@
|
|||||||
#include <DB/Interpreters/Settings.h>
|
#include <DB/Interpreters/Settings.h>
|
||||||
#include <DB/Interpreters/Users.h>
|
#include <DB/Interpreters/Users.h>
|
||||||
#include <DB/Interpreters/Quota.h>
|
#include <DB/Interpreters/Quota.h>
|
||||||
#include <DB/Interpreters/Dictionaries.h>
|
#include <DB/Interpreters/EmbeddedDictionaries.h>
|
||||||
#include <DB/Interpreters/ExternalDictionaries.h>
|
#include <DB/Interpreters/ExternalDictionaries.h>
|
||||||
#include <DB/Interpreters/ProcessList.h>
|
#include <DB/Interpreters/ProcessList.h>
|
||||||
#include <DB/Interpreters/Cluster.h>
|
#include <DB/Interpreters/Cluster.h>
|
||||||
@ -83,7 +84,8 @@ struct ContextShared
|
|||||||
|
|
||||||
/// For access of most of shared objects. Recursive mutex.
|
/// For access of most of shared objects. Recursive mutex.
|
||||||
mutable Poco::Mutex mutex;
|
mutable Poco::Mutex mutex;
|
||||||
/// Separate mutex for access of external dictionaries. Separate mutex to avoid locks when server doing request to itself.
|
/// Separate mutex for access of dictionaries. Separate mutex to avoid locks when server doing request to itself.
|
||||||
|
mutable std::mutex embedded_dictionaries_mutex;
|
||||||
mutable std::mutex external_dictionaries_mutex;
|
mutable std::mutex external_dictionaries_mutex;
|
||||||
/// Separate mutex for re-initialization of zookeer session. This operation could take a long time and must not interfere with another operations.
|
/// Separate mutex for re-initialization of zookeer session. This operation could take a long time and must not interfere with another operations.
|
||||||
mutable std::mutex zookeeper_mutex;
|
mutable std::mutex zookeeper_mutex;
|
||||||
@ -100,7 +102,7 @@ struct ContextShared
|
|||||||
TableFunctionFactory table_function_factory; /// Табличные функции.
|
TableFunctionFactory table_function_factory; /// Табличные функции.
|
||||||
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
|
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
|
||||||
FormatFactory format_factory; /// Форматы.
|
FormatFactory format_factory; /// Форматы.
|
||||||
mutable std::shared_ptr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво.
|
mutable std::shared_ptr<EmbeddedDictionaries> embedded_dictionaries; /// Словари Метрики. Инициализируются лениво.
|
||||||
mutable std::shared_ptr<ExternalDictionaries> external_dictionaries;
|
mutable std::shared_ptr<ExternalDictionaries> external_dictionaries;
|
||||||
Users users; /// Известные пользователи.
|
Users users; /// Известные пользователи.
|
||||||
Quotas quotas; /// Известные квоты на использование ресурсов.
|
Quotas quotas; /// Известные квоты на использование ресурсов.
|
||||||
@ -788,9 +790,9 @@ Context & Context::getGlobalContext()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
const Dictionaries & Context::getDictionaries() const
|
const EmbeddedDictionaries & Context::getEmbeddedDictionaries() const
|
||||||
{
|
{
|
||||||
return getDictionariesImpl(false);
|
return getEmbeddedDictionariesImpl(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -800,14 +802,14 @@ const ExternalDictionaries & Context::getExternalDictionaries() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
const Dictionaries & Context::getDictionariesImpl(const bool throw_on_error) const
|
const EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const
|
||||||
{
|
{
|
||||||
auto lock = getLock();
|
std::lock_guard<std::mutex> lock(shared->embedded_dictionaries_mutex);
|
||||||
|
|
||||||
if (!shared->dictionaries)
|
if (!shared->embedded_dictionaries)
|
||||||
shared->dictionaries = std::make_shared<Dictionaries>(throw_on_error);
|
shared->embedded_dictionaries = std::make_shared<EmbeddedDictionaries>(throw_on_error);
|
||||||
|
|
||||||
return *shared->dictionaries;
|
return *shared->embedded_dictionaries;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -826,9 +828,9 @@ const ExternalDictionaries & Context::getExternalDictionariesImpl(const bool thr
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void Context::tryCreateDictionaries() const
|
void Context::tryCreateEmbeddedDictionaries() const
|
||||||
{
|
{
|
||||||
static_cast<void>(getDictionariesImpl(true));
|
static_cast<void>(getEmbeddedDictionariesImpl(true));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
121
dbms/src/Interpreters/EmbeddedDictionaries.cpp
Normal file
121
dbms/src/Interpreters/EmbeddedDictionaries.cpp
Normal file
@ -0,0 +1,121 @@
|
|||||||
|
#include <common/logger_useful.h>
|
||||||
|
#include <Poco/Util/Application.h>
|
||||||
|
#include <DB/Interpreters/EmbeddedDictionaries.h>
|
||||||
|
#include <DB/Dictionaries/Embedded/RegionsHierarchies.h>
|
||||||
|
#include <DB/Dictionaries/Embedded/TechDataHierarchy.h>
|
||||||
|
#include <DB/Dictionaries/Embedded/RegionsNames.h>
|
||||||
|
#include <DB/Common/setThreadName.h>
|
||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void EmbeddedDictionaries::handleException(const bool throw_on_error) const
|
||||||
|
{
|
||||||
|
const auto exception_ptr = std::current_exception();
|
||||||
|
|
||||||
|
tryLogCurrentException(log, "Cannot load dictionary! You must resolve this manually.");
|
||||||
|
|
||||||
|
if (throw_on_error)
|
||||||
|
std::rethrow_exception(exception_ptr);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <typename Dictionary>
|
||||||
|
bool EmbeddedDictionaries::reloadDictionary(MultiVersion<Dictionary> & dictionary, const bool throw_on_error)
|
||||||
|
{
|
||||||
|
if (Dictionary::isConfigured() && (!is_fast_start_stage || !dictionary.get()))
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
auto new_dictionary = std::make_unique<Dictionary>();
|
||||||
|
new_dictionary->reload();
|
||||||
|
dictionary.set(new_dictionary.release());
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
handleException(throw_on_error);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool EmbeddedDictionaries::reloadImpl(const bool throw_on_error)
|
||||||
|
{
|
||||||
|
/** Если не удаётся обновить справочники, то несмотря на это, не кидаем исключение (используем старые справочники).
|
||||||
|
* Если старых корректных справочников нет, то при использовании функций, которые от них зависят,
|
||||||
|
* будет кидаться исключение.
|
||||||
|
* Производится попытка загрузить каждый справочник по-отдельности.
|
||||||
|
*/
|
||||||
|
|
||||||
|
LOG_INFO(log, "Loading dictionaries.");
|
||||||
|
|
||||||
|
bool was_exception = false;
|
||||||
|
|
||||||
|
if (!reloadDictionary<TechDataHierarchy>(tech_data_hierarchy, throw_on_error))
|
||||||
|
was_exception = true;
|
||||||
|
|
||||||
|
if (!reloadDictionary<RegionsHierarchies>(regions_hierarchies, throw_on_error))
|
||||||
|
was_exception = true;
|
||||||
|
|
||||||
|
if (!reloadDictionary<RegionsNames>(regions_names, throw_on_error))
|
||||||
|
was_exception = true;
|
||||||
|
|
||||||
|
if (!was_exception)
|
||||||
|
LOG_INFO(log, "Loaded dictionaries.");
|
||||||
|
|
||||||
|
return !was_exception;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void EmbeddedDictionaries::reloadPeriodically()
|
||||||
|
{
|
||||||
|
setThreadName("DictReload");
|
||||||
|
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
if (destroy.tryWait(cur_reload_period * 1000))
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (reloadImpl(false))
|
||||||
|
{
|
||||||
|
/// Success
|
||||||
|
cur_reload_period = reload_period;
|
||||||
|
is_fast_start_stage = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (is_fast_start_stage)
|
||||||
|
{
|
||||||
|
cur_reload_period = std::min(reload_period, 2 * cur_reload_period); /// exponentially increase delay
|
||||||
|
is_fast_start_stage = cur_reload_period < reload_period; /// leave fast start state
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
EmbeddedDictionaries::EmbeddedDictionaries(const bool throw_on_error, const int reload_period_)
|
||||||
|
: reload_period(reload_period_), log(&Logger::get("EmbeddedDictionaries"))
|
||||||
|
{
|
||||||
|
reloadImpl(throw_on_error);
|
||||||
|
reloading_thread = std::thread([this] { reloadPeriodically(); });
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
EmbeddedDictionaries::EmbeddedDictionaries(const bool throw_on_error)
|
||||||
|
: EmbeddedDictionaries(throw_on_error,
|
||||||
|
Poco::Util::Application::instance().config().getInt("builtin_dictionaries_reload_interval", 3600))
|
||||||
|
{}
|
||||||
|
|
||||||
|
|
||||||
|
EmbeddedDictionaries::~EmbeddedDictionaries()
|
||||||
|
{
|
||||||
|
destroy.set();
|
||||||
|
reloading_thread.join();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -6,6 +6,7 @@
|
|||||||
#include <ext/scope_guard.hpp>
|
#include <ext/scope_guard.hpp>
|
||||||
#include <Poco/Util/Application.h>
|
#include <Poco/Util/Application.h>
|
||||||
#include <Poco/Glob.h>
|
#include <Poco/Glob.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
|
@ -482,7 +482,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
|||||||
String as_table_name = create.as_table;
|
String as_table_name = create.as_table;
|
||||||
|
|
||||||
StoragePtr as_storage;
|
StoragePtr as_storage;
|
||||||
IStorage::TableStructureReadLockPtr as_storage_lock;
|
TableStructureReadLockPtr as_storage_lock;
|
||||||
if (!as_table_name.empty())
|
if (!as_table_name.empty())
|
||||||
{
|
{
|
||||||
as_storage = context.getTable(as_database_name, as_table_name);
|
as_storage = context.getTable(as_database_name, as_table_name);
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#include <DB/Parsers/ASTRenameQuery.h>
|
#include <DB/Parsers/ASTRenameQuery.h>
|
||||||
#include <DB/Databases/IDatabase.h>
|
#include <DB/Databases/IDatabase.h>
|
||||||
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
@ -89,7 +90,7 @@ BlockIO InterpreterRenameQuery::execute()
|
|||||||
"Some table right now is being renamed to " + to.database_name + "." + to.table_name));
|
"Some table right now is being renamed to " + to.database_name + "." + to.table_name));
|
||||||
}
|
}
|
||||||
|
|
||||||
std::vector<IStorage::TableFullWriteLockPtr> locks;
|
std::vector<TableFullWriteLockPtr> locks;
|
||||||
locks.reserve(unique_tables_from.size());
|
locks.reserve(unique_tables_from.size());
|
||||||
|
|
||||||
for (const auto & names : unique_tables_from)
|
for (const auto & names : unique_tables_from)
|
||||||
|
@ -31,6 +31,9 @@
|
|||||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||||
#include <DB/Interpreters/InterpreterSetQuery.h>
|
#include <DB/Interpreters/InterpreterSetQuery.h>
|
||||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||||
|
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
|
|
||||||
#include <DB/TableFunctions/ITableFunction.h>
|
#include <DB/TableFunctions/ITableFunction.h>
|
||||||
#include <DB/TableFunctions/TableFunctionFactory.h>
|
#include <DB/TableFunctions/TableFunctionFactory.h>
|
||||||
|
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
#include <DB/Columns/ColumnFixedString.h>
|
#include <DB/Columns/ColumnFixedString.h>
|
||||||
|
|
||||||
@ -19,6 +21,19 @@ namespace ErrorCodes
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Join::Join(const Names & key_names_left_, const Names & key_names_right_,
|
||||||
|
const Limits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_)
|
||||||
|
: kind(kind_), strictness(strictness_),
|
||||||
|
key_names_left(key_names_left_),
|
||||||
|
key_names_right(key_names_right_),
|
||||||
|
log(&Logger::get("Join")),
|
||||||
|
max_rows(limits.max_rows_in_join),
|
||||||
|
max_bytes(limits.max_bytes_in_join),
|
||||||
|
overflow_mode(limits.join_overflow_mode)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
Join::Type Join::chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes)
|
Join::Type Join::chooseMethod(const ConstColumnPlainPtrs & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes)
|
||||||
{
|
{
|
||||||
size_t keys_size = key_columns.size();
|
size_t keys_size = key_columns.size();
|
||||||
|
@ -1,4 +1,8 @@
|
|||||||
#include <DB/Interpreters/ProcessList.h>
|
#include <DB/Interpreters/ProcessList.h>
|
||||||
|
#include <DB/Interpreters/Settings.h>
|
||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -516,7 +516,7 @@ int Server::main(const std::vector<std::string> & args)
|
|||||||
{
|
{
|
||||||
if (!config().getBool("dictionaries_lazy_load", true))
|
if (!config().getBool("dictionaries_lazy_load", true))
|
||||||
{
|
{
|
||||||
global_context->tryCreateDictionaries();
|
global_context->tryCreateEmbeddedDictionaries();
|
||||||
global_context->tryCreateExternalDictionaries();
|
global_context->tryCreateExternalDictionaries();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
16
dbms/src/Storages/IStorage.cpp
Normal file
16
dbms/src/Storages/IStorage.cpp
Normal file
@ -0,0 +1,16 @@
|
|||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
TableStructureReadLock::TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data)
|
||||||
|
: storage(storage_)
|
||||||
|
{
|
||||||
|
if (lock_data)
|
||||||
|
data_lock.emplace(storage->data_lock);
|
||||||
|
if (lock_structure)
|
||||||
|
structure_lock.emplace(storage->structure_lock);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -3,6 +3,7 @@
|
|||||||
#include <DB/Common/CurrentMetrics.h>
|
#include <DB/Common/CurrentMetrics.h>
|
||||||
#include <DB/Common/NetException.h>
|
#include <DB/Common/NetException.h>
|
||||||
#include <DB/IO/ReadBufferFromHTTP.h>
|
#include <DB/IO/ReadBufferFromHTTP.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
|
||||||
namespace CurrentMetrics
|
namespace CurrentMetrics
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <DB/Storages/MergeTree/MergeTreeBlockInputStream.h>
|
#include <DB/Storages/MergeTree/MergeTreeBlockInputStream.h>
|
||||||
#include <DB/Storages/MergeTree/MergedBlockOutputStream.h>
|
#include <DB/Storages/MergeTree/MergedBlockOutputStream.h>
|
||||||
#include <DB/Storages/MergeTree/MergeTreePartChecker.h>
|
#include <DB/Storages/MergeTree/MergeTreePartChecker.h>
|
||||||
|
#include <DB/Storages/AlterCommands.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/ASTNameTypePair.h>
|
#include <DB/Parsers/ASTNameTypePair.h>
|
||||||
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
#include <DB/DataStreams/ExpressionBlockInputStream.h>
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <DB/Common/escapeForFileName.h>
|
#include <DB/Common/escapeForFileName.h>
|
||||||
#include <DB/DataTypes/DataTypeArray.h>
|
#include <DB/DataTypes/DataTypeArray.h>
|
||||||
#include <DB/IO/HashingWriteBuffer.h>
|
#include <DB/IO/HashingWriteBuffer.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
|
||||||
namespace ProfileEvents
|
namespace ProfileEvents
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
#include <DB/Columns/ColumnsNumber.h>
|
#include <DB/Columns/ColumnsNumber.h>
|
||||||
#include <DB/Common/CurrentMetrics.h>
|
#include <DB/Common/CurrentMetrics.h>
|
||||||
#include <DB/Common/escapeForFileName.h>
|
#include <DB/Common/escapeForFileName.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
|
||||||
namespace CurrentMetrics
|
namespace CurrentMetrics
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
#include <DB/Interpreters/evaluateMissingDefaults.h>
|
#include <DB/Interpreters/evaluateMissingDefaults.h>
|
||||||
#include <DB/Storages/MergeTree/MergeTreeReader.h>
|
#include <DB/Storages/MergeTree/MergeTreeReader.h>
|
||||||
#include <DB/Columns/ColumnNullable.h>
|
#include <DB/Columns/ColumnNullable.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -6,8 +6,9 @@
|
|||||||
#include <DB/DataTypes/DataTypeNullable.h>
|
#include <DB/DataTypes/DataTypeNullable.h>
|
||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
#include <DB/Columns/ColumnNullable.h>
|
#include <DB/Columns/ColumnNullable.h>
|
||||||
|
|
||||||
#include <DB/Common/StringUtils.h>
|
#include <DB/Common/StringUtils.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -4,6 +4,8 @@
|
|||||||
#include <DB/IO/ReadBufferFromHTTP.h>
|
#include <DB/IO/ReadBufferFromHTTP.h>
|
||||||
#include <DB/IO/ReadHelpers.h>
|
#include <DB/IO/ReadHelpers.h>
|
||||||
#include <DB/IO/WriteHelpers.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -3,8 +3,8 @@
|
|||||||
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
||||||
#include <DB/IO/ReadHelpers.h>
|
#include <DB/IO/ReadHelpers.h>
|
||||||
#include <DB/IO/WriteHelpers.h>
|
#include <DB/IO/WriteHelpers.h>
|
||||||
|
|
||||||
#include <DB/IO/InterserverWriteBuffer.h>
|
#include <DB/IO/InterserverWriteBuffer.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
|
|
||||||
namespace CurrentMetrics
|
namespace CurrentMetrics
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
#include <DB/Parsers/ASTExpressionList.h>
|
#include <DB/Parsers/ASTExpressionList.h>
|
||||||
#include <DB/Common/setThreadName.h>
|
#include <DB/Common/setThreadName.h>
|
||||||
#include <DB/Common/CurrentMetrics.h>
|
#include <DB/Common/CurrentMetrics.h>
|
||||||
|
#include <common/logger_useful.h>
|
||||||
#include <Poco/Ext/ThreadNumber.h>
|
#include <Poco/Ext/ThreadNumber.h>
|
||||||
|
|
||||||
#include <ext/range.hpp>
|
#include <ext/range.hpp>
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#include <DB/Storages/StorageJoin.h>
|
#include <DB/Storages/StorageJoin.h>
|
||||||
#include <DB/Interpreters/Join.h>
|
#include <DB/Interpreters/Join.h>
|
||||||
|
#include <DB/Interpreters/Limits.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -22,6 +22,8 @@
|
|||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
#include <DB/Columns/ColumnNullable.h>
|
#include <DB/Columns/ColumnNullable.h>
|
||||||
|
|
||||||
|
#include <DB/Interpreters/Settings.h>
|
||||||
|
|
||||||
#include <Poco/Path.h>
|
#include <Poco/Path.h>
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
|
|
||||||
|
@ -125,7 +125,7 @@ BlockInputStreams StorageMerge::read(
|
|||||||
if (!table->supportsPrewhere())
|
if (!table->supportsPrewhere())
|
||||||
throw Exception("Storage " + table->getName() + " doesn't support PREWHERE.", ErrorCodes::ILLEGAL_PREWHERE);
|
throw Exception("Storage " + table->getName() + " doesn't support PREWHERE.", ErrorCodes::ILLEGAL_PREWHERE);
|
||||||
|
|
||||||
TableLocks table_locks;
|
TableStructureReadLocks table_locks;
|
||||||
|
|
||||||
/// Нельзя, чтобы эти таблицы кто-нибудь удалил, пока мы их читаем.
|
/// Нельзя, чтобы эти таблицы кто-нибудь удалил, пока мы их читаем.
|
||||||
for (auto & table : selected_tables)
|
for (auto & table : selected_tables)
|
||||||
|
@ -29,6 +29,8 @@
|
|||||||
|
|
||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
|
|
||||||
|
#include <DB/Interpreters/Settings.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageStripeLog.h>
|
#include <DB/Storages/StorageStripeLog.h>
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
|
|
||||||
|
@ -28,6 +28,8 @@
|
|||||||
#include <DB/Columns/ColumnArray.h>
|
#include <DB/Columns/ColumnArray.h>
|
||||||
#include <DB/Columns/ColumnNullable.h>
|
#include <DB/Columns/ColumnNullable.h>
|
||||||
|
|
||||||
|
#include <DB/Interpreters/Settings.h>
|
||||||
|
|
||||||
#include <DB/Storages/StorageTinyLog.h>
|
#include <DB/Storages/StorageTinyLog.h>
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
|
|
||||||
|
@ -126,7 +126,7 @@ BlockInputStreams StorageSystemColumns::read(
|
|||||||
|
|
||||||
{
|
{
|
||||||
StoragePtr storage = storages.at(std::make_pair(database_name, table_name));
|
StoragePtr storage = storages.at(std::make_pair(database_name, table_name));
|
||||||
IStorage::TableStructureReadLockPtr table_lock;
|
TableStructureReadLockPtr table_lock;
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
|
@ -172,7 +172,7 @@ BlockInputStreams StorageSystemParts::read(
|
|||||||
}
|
}
|
||||||
|
|
||||||
StoragePtr storage = storages.at(std::make_pair(database, table));
|
StoragePtr storage = storages.at(std::make_pair(database, table));
|
||||||
IStorage::TableStructureReadLockPtr table_lock;
|
TableStructureReadLockPtr table_lock;
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||||
|
#include <DB/Interpreters/Settings.h>
|
||||||
#include <DB/Storages/System/StorageSystemSettings.h>
|
#include <DB/Storages/System/StorageSystemSettings.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,3 +1,6 @@
|
|||||||
|
#include <DB/Common/Exception.h>
|
||||||
|
#include <DB/Interpreters/Context.h>
|
||||||
|
|
||||||
#include <DB/TableFunctions/TableFunctionMerge.h>
|
#include <DB/TableFunctions/TableFunctionMerge.h>
|
||||||
#include <DB/TableFunctions/TableFunctionRemote.h>
|
#include <DB/TableFunctions/TableFunctionRemote.h>
|
||||||
#include <DB/TableFunctions/TableFunctionShardByHash.h>
|
#include <DB/TableFunctions/TableFunctionShardByHash.h>
|
||||||
@ -16,7 +19,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
|
|
||||||
TableFunctionPtr TableFunctionFactory::get(
|
TableFunctionPtr TableFunctionFactory::get(
|
||||||
const String & name,
|
const std::string & name,
|
||||||
const Context & context) const
|
const Context & context) const
|
||||||
{
|
{
|
||||||
if (context.getSettings().limits.readonly == 1) /** Например, для readonly = 2 - разрешено. */
|
if (context.getSettings().limits.readonly == 1) /** Например, для readonly = 2 - разрешено. */
|
||||||
|
@ -4,8 +4,10 @@
|
|||||||
#include <DB/Parsers/ASTExpressionList.h>
|
#include <DB/Parsers/ASTExpressionList.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/ASTLiteral.h>
|
#include <DB/Parsers/ASTLiteral.h>
|
||||||
|
#include <DB/Parsers/ASTFunction.h>
|
||||||
#include <DB/TableFunctions/ITableFunction.h>
|
#include <DB/TableFunctions/ITableFunction.h>
|
||||||
#include <DB/Interpreters/evaluateConstantExpression.h>
|
#include <DB/Interpreters/evaluateConstantExpression.h>
|
||||||
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <DB/Databases/IDatabase.h>
|
#include <DB/Databases/IDatabase.h>
|
||||||
#include <DB/TableFunctions/TableFunctionMerge.h>
|
#include <DB/TableFunctions/TableFunctionMerge.h>
|
||||||
|
|
||||||
|
@ -2,8 +2,11 @@
|
|||||||
#include <DB/Storages/StorageDistributed.h>
|
#include <DB/Storages/StorageDistributed.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/ASTLiteral.h>
|
#include <DB/Parsers/ASTLiteral.h>
|
||||||
|
#include <DB/Parsers/ASTFunction.h>
|
||||||
|
#include <DB/Parsers/ASTExpressionList.h>
|
||||||
#include <DB/Interpreters/evaluateConstantExpression.h>
|
#include <DB/Interpreters/evaluateConstantExpression.h>
|
||||||
#include <DB/Interpreters/Cluster.h>
|
#include <DB/Interpreters/Cluster.h>
|
||||||
|
#include <DB/Interpreters/Context.h>
|
||||||
|
|
||||||
#include <DB/TableFunctions/TableFunctionRemote.h>
|
#include <DB/TableFunctions/TableFunctionRemote.h>
|
||||||
|
|
||||||
|
@ -2,8 +2,10 @@
|
|||||||
#include <DB/Storages/StorageDistributed.h>
|
#include <DB/Storages/StorageDistributed.h>
|
||||||
#include <DB/Parsers/ASTIdentifier.h>
|
#include <DB/Parsers/ASTIdentifier.h>
|
||||||
#include <DB/Parsers/ASTLiteral.h>
|
#include <DB/Parsers/ASTLiteral.h>
|
||||||
|
#include <DB/Parsers/ASTFunction.h>
|
||||||
#include <DB/Interpreters/evaluateConstantExpression.h>
|
#include <DB/Interpreters/evaluateConstantExpression.h>
|
||||||
#include <DB/Interpreters/Cluster.h>
|
#include <DB/Interpreters/Cluster.h>
|
||||||
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <DB/Interpreters/getClusterName.h>
|
#include <DB/Interpreters/getClusterName.h>
|
||||||
#include <DB/Common/SipHash.h>
|
#include <DB/Common/SipHash.h>
|
||||||
#include <DB/TableFunctions/TableFunctionShardByHash.h>
|
#include <DB/TableFunctions/TableFunctionShardByHash.h>
|
||||||
|
@ -2,6 +2,8 @@
|
|||||||
#include <DB/Interpreters/Context.h>
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
||||||
#include <DB/DataTypes/DataTypeFactory.h>
|
#include <DB/DataTypes/DataTypeFactory.h>
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
|
#include <DB/Parsers/IAST.h>
|
||||||
|
|
||||||
#include <DB/TableFunctions/getStructureOfRemoteTable.h>
|
#include <DB/TableFunctions/getStructureOfRemoteTable.h>
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user