From 8149d2ced373cd557dd4afcad9c2c504d9744cbc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 04:46:41 +0400 Subject: [PATCH 01/23] Fixed build [#METR-10384]. --- dbms/include/DB/Common/LRUCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/include/DB/Common/LRUCache.h b/dbms/include/DB/Common/LRUCache.h index 6bf4743d548..0f840e6fe6e 100644 --- a/dbms/include/DB/Common/LRUCache.h +++ b/dbms/include/DB/Common/LRUCache.h @@ -135,7 +135,7 @@ private: size_t current_size = 0; const size_t max_size; - Poco::FastMutex mutex; + mutable Poco::FastMutex mutex; size_t hits = 0; size_t misses = 0; From 8c87b14c7f832228fcf74cacfce0245e82fd07db Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 08:17:17 +0400 Subject: [PATCH 02/23] Fixed warnings on clang [#METR-2807]. --- dbms/include/DB/Columns/IColumnDummy.h | 2 +- dbms/include/DB/Common/LRUCache.h | 2 +- dbms/include/DB/Core/Protocol.h | 4 ++-- dbms/include/DB/Core/QueryProcessingStage.h | 2 +- dbms/include/DB/IO/RemoteReadBuffer.h | 4 ++-- dbms/include/DB/Interpreters/HashSet.h | 8 ++++++-- dbms/include/DB/Interpreters/SplittingAggregator.h | 1 - dbms/include/DB/Storages/StorageDistributed.h | 4 ---- dbms/include/DB/TableFunctions/TableFunctionRemote.h | 2 +- dbms/src/IO/tests/parse_int_perf.cpp | 9 ++++++--- dbms/src/Storages/StorageDistributed.cpp | 8 ++------ dbms/src/Storages/StorageFactory.cpp | 2 +- 12 files changed, 23 insertions(+), 25 deletions(-) diff --git a/dbms/include/DB/Columns/IColumnDummy.h b/dbms/include/DB/Columns/IColumnDummy.h index 6e094a90283..ffa6a8742c5 100644 --- a/dbms/include/DB/Columns/IColumnDummy.h +++ b/dbms/include/DB/Columns/IColumnDummy.h @@ -18,7 +18,7 @@ public: virtual ColumnPtr cloneDummy(size_t s_) const = 0; ColumnPtr cloneResized(size_t s_) const { return cloneDummy(s_); } - bool isConst() { return true; } + bool isConst() const { return true; } size_t size() const { return s; } void insertDefault() { ++s; } size_t byteSize() const { return 0; } diff --git a/dbms/include/DB/Common/LRUCache.h b/dbms/include/DB/Common/LRUCache.h index 0f840e6fe6e..f8575891085 100644 --- a/dbms/include/DB/Common/LRUCache.h +++ b/dbms/include/DB/Common/LRUCache.h @@ -85,7 +85,7 @@ public: removeOverflow(); } - void getStats(size_t & out_hits, size_t & out_misses) const volatile + void getStats(size_t & out_hits, size_t & out_misses) const { Poco::ScopedLock lock(mutex); /// Синхронизация не нужна. diff --git a/dbms/include/DB/Core/Protocol.h b/dbms/include/DB/Core/Protocol.h index 8bcb5822da5..7dadb17f953 100644 --- a/dbms/include/DB/Core/Protocol.h +++ b/dbms/include/DB/Core/Protocol.h @@ -73,7 +73,7 @@ namespace Protocol inline const char * toString(UInt64 packet) { static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", "Extremes" }; - return packet >= 0 && packet < 9 + return packet < 9 ? data[packet] : "Unknown packet"; } @@ -97,7 +97,7 @@ namespace Protocol inline const char * toString(UInt64 packet) { static const char * data[] = { "Hello", "Query", "Data", "Cancel", "Ping" }; - return packet >= 0 && packet < 5 + return packet < 5 ? data[packet] : "Unknown packet"; } diff --git a/dbms/include/DB/Core/QueryProcessingStage.h b/dbms/include/DB/Core/QueryProcessingStage.h index 2e3097a9f57..f706ef6b658 100644 --- a/dbms/include/DB/Core/QueryProcessingStage.h +++ b/dbms/include/DB/Core/QueryProcessingStage.h @@ -20,7 +20,7 @@ namespace QueryProcessingStage inline const char * toString(UInt64 stage) { static const char * data[] = { "FetchColumns", "WithMergeableState", "Complete" }; - return stage >= 0 && stage < 3 + return stage < 3 ? data[stage] : "Unknown stage"; } diff --git a/dbms/include/DB/IO/RemoteReadBuffer.h b/dbms/include/DB/IO/RemoteReadBuffer.h index 96b7669a32b..5f35211f118 100644 --- a/dbms/include/DB/IO/RemoteReadBuffer.h +++ b/dbms/include/DB/IO/RemoteReadBuffer.h @@ -51,7 +51,7 @@ public: session.setPort(port); /// устанавливаем таймаут - session.setTimeout(Poco::Timespan(timeout_ || DEFAULT_REMOTE_READ_BUFFER_TIMEOUT, 0)); + session.setTimeout(Poco::Timespan(timeout_ ? timeout_ : DEFAULT_REMOTE_READ_BUFFER_TIMEOUT, 0)); Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, uri.str()); Poco::Net::HTTPResponse response; @@ -100,7 +100,7 @@ public: Poco::Net::HTTPClientSession session; session.setHost(host); session.setPort(port); - session.setTimeout(Poco::Timespan(timeout_ || DEFAULT_REMOTE_READ_BUFFER_TIMEOUT, 0)); + session.setTimeout(Poco::Timespan(timeout_ ? timeout_ : DEFAULT_REMOTE_READ_BUFFER_TIMEOUT, 0)); Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, uri.str()); Poco::Net::HTTPResponse response; diff --git a/dbms/include/DB/Interpreters/HashSet.h b/dbms/include/DB/Interpreters/HashSet.h index 93c024eb78f..67f4c52d494 100644 --- a/dbms/include/DB/Interpreters/HashSet.h +++ b/dbms/include/DB/Interpreters/HashSet.h @@ -417,7 +417,9 @@ public: size_degree = new_size <= 1 ? GrowthTraits::INITIAL_SIZE_DEGREE - : std::max(GrowthTraits::INITIAL_SIZE_DEGREE, static_cast(log2(new_size - 1)) + 2); + : ((GrowthTraits::INITIAL_SIZE_DEGREE > static_cast(log2(new_size - 1)) + 2) + ? GrowthTraits::INITIAL_SIZE_DEGREE + : (static_cast(log2(new_size - 1)) + 2)); alloc(); @@ -436,7 +438,9 @@ public: size_t new_size_degree = new_size <= 1 ? GrowthTraits::INITIAL_SIZE_DEGREE - : std::max(GrowthTraits::INITIAL_SIZE_DEGREE, static_cast(log2(new_size - 1)) + 2); + : ((GrowthTraits::INITIAL_SIZE_DEGREE > static_cast(log2(new_size - 1)) + 2) + ? GrowthTraits::INITIAL_SIZE_DEGREE + : (static_cast(log2(new_size - 1)) + 2)); if (new_size_degree > size_degree) resize(new_size_degree); diff --git a/dbms/include/DB/Interpreters/SplittingAggregator.h b/dbms/include/DB/Interpreters/SplittingAggregator.h index e0059f6fe18..b4d7fd118d9 100644 --- a/dbms/include/DB/Interpreters/SplittingAggregator.h +++ b/dbms/include/DB/Interpreters/SplittingAggregator.h @@ -86,7 +86,6 @@ private: Sizes key_sizes; - UInt128Hash hash_func_128; StringRefHash hash_func_string; /// Для более точного контроля max_rows_to_group_by. diff --git a/dbms/include/DB/Storages/StorageDistributed.h b/dbms/include/DB/Storages/StorageDistributed.h index aa55b80ba81..2ffa9ce2791 100644 --- a/dbms/include/DB/Storages/StorageDistributed.h +++ b/dbms/include/DB/Storages/StorageDistributed.h @@ -25,7 +25,6 @@ public: const String & remote_database_, /// БД на удалённых серверах. const String & remote_table_, /// Имя таблицы на удалённых серверах. const String & cluster_name, - const DataTypeFactory & data_type_factory_, Context & context_, const String & sign_column_name_ = ""); @@ -35,7 +34,6 @@ public: const String & remote_database_, /// БД на удалённых серверах. const String & remote_table_, /// Имя таблицы на удалённых серверах. SharedPtr & owned_cluster_, - const DataTypeFactory & data_type_factory_, Context & context_, const String & sign_column_name_ = ""); @@ -75,7 +73,6 @@ private: const String & remote_database_, const String & remote_table_, Cluster & cluster_, - const DataTypeFactory & data_type_factory_, const Context & context_, const String & sign_column_name_ = ""); @@ -86,7 +83,6 @@ private: NamesAndTypesListPtr columns; String remote_database; String remote_table; - const DataTypeFactory & data_type_factory; String sign_column_name; /// Имя виртуального столбца, куда записывается имя хоста (Например "_host"). diff --git a/dbms/include/DB/TableFunctions/TableFunctionRemote.h b/dbms/include/DB/TableFunctions/TableFunctionRemote.h index d8096d3f42e..a4303eec87b 100644 --- a/dbms/include/DB/TableFunctions/TableFunctionRemote.h +++ b/dbms/include/DB/TableFunctions/TableFunctionRemote.h @@ -62,7 +62,7 @@ public: SharedPtr cluster = new Cluster(context.getSettings(), context.getDataTypeFactory(), names, username, password); return StorageDistributed::create(getName(), chooseColumns(*cluster, remote_database, remote_table, context), - remote_database, remote_table, cluster, context.getDataTypeFactory(), context); + remote_database, remote_table, cluster, context); } private: diff --git a/dbms/src/IO/tests/parse_int_perf.cpp b/dbms/src/IO/tests/parse_int_perf.cpp index 9e7064a224f..f3a539f4ac5 100644 --- a/dbms/src/IO/tests/parse_int_perf.cpp +++ b/dbms/src/IO/tests/parse_int_perf.cpp @@ -17,7 +17,9 @@ #include -const UInt32 decimal_table[10000] = +#if 0 + +static const UInt32 decimal_table[10000] = { 0x30303030,0x31303030,0x32303030,0x33303030,0x34303030,0x35303030,0x36303030,0x37303030,0x38303030,0x39303030, 0x30313030,0x31313030,0x32313030,0x33313030,0x34313030,0x35313030,0x36313030,0x37313030,0x38313030,0x39313030, @@ -1022,7 +1024,7 @@ const UInt32 decimal_table[10000] = }; -const UInt8 length_table[10000] = +static const UInt8 length_table[10000] = { 3,3,3,3,3,3,3,3,3,3,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2, 2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2, @@ -1095,7 +1097,6 @@ void writeUIntTextTable(T x, DB::WriteBuffer & buf) buf.write(data.chars + length_table[x], len - length_table[x]); } - template void writeIntTextTable(T x, DB::WriteBuffer & buf) { @@ -1127,6 +1128,8 @@ void writeIntTextTable(T x, DB::WriteBuffer & buf) writeUIntTextTable(static_cast::type>(x), buf); } +#endif + UInt64 rdtsc() { diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 26ec860183f..ede4fb735ad 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -21,12 +21,10 @@ StorageDistributed::StorageDistributed( const String & remote_database_, const String & remote_table_, Cluster & cluster_, - const DataTypeFactory & data_type_factory_, const Context & context_, const String & sign_column_name_) : name(name_), columns(columns_), remote_database(remote_database_), remote_table(remote_table_), - data_type_factory(data_type_factory_), sign_column_name(sign_column_name_), context(context_), cluster(cluster_) @@ -45,12 +43,11 @@ StoragePtr StorageDistributed::create( const String & remote_database_, const String & remote_table_, const String & cluster_name, - const DataTypeFactory & data_type_factory_, Context & context_, const String & sign_column_name_) { context_.initClusters(); - return (new StorageDistributed(name_, columns_, remote_database_, remote_table_, context_.getCluster(cluster_name), data_type_factory_, context_, sign_column_name_))->thisPtr(); + return (new StorageDistributed(name_, columns_, remote_database_, remote_table_, context_.getCluster(cluster_name), context_, sign_column_name_))->thisPtr(); } @@ -60,11 +57,10 @@ StoragePtr StorageDistributed::create( const String & remote_database_, const String & remote_table_, SharedPtr & owned_cluster_, - const DataTypeFactory & data_type_factory_, Context & context_, const String & sign_column_name_) { - auto res = new StorageDistributed(name_, columns_, remote_database_, remote_table_, *owned_cluster_, data_type_factory_, context_, sign_column_name_); + auto res = new StorageDistributed(name_, columns_, remote_database_, remote_table_, *owned_cluster_, context_, sign_column_name_); /// Захватываем владение объектом-кластером. res->owned_cluster = owned_cluster_; diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 42077e75b9c..08cb6904183 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -149,7 +149,7 @@ StoragePtr StorageFactory::get( String sign_column_name = args.size() == 4 ? dynamic_cast(*args[3]).name : ""; return StorageDistributed::create(table_name, columns, remote_database, remote_table, cluster_name, - context.getDataTypeFactory(), context, sign_column_name); + context, sign_column_name); } else if (name == "MergeTree" || name == "SummingMergeTree") { From a4778d81cddb65f44aa8c0d7e550aa0fd5548a82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 13:33:18 +0400 Subject: [PATCH 03/23] dbms: added trivial benchmark tool [#METR-2944]. --- .../DB/DataStreams/IBlockInputStream.h | 2 +- dbms/src/Client/Benchmark.cpp | 313 ++++++++++++++++++ dbms/src/Client/Client.cpp | 71 +--- dbms/src/Client/InterruptListener.h | 79 +++++ 4 files changed, 395 insertions(+), 70 deletions(-) create mode 100644 dbms/src/Client/Benchmark.cpp create mode 100644 dbms/src/Client/InterruptListener.h diff --git a/dbms/include/DB/DataStreams/IBlockInputStream.h b/dbms/include/DB/DataStreams/IBlockInputStream.h index b5fda965f8f..db49955c310 100644 --- a/dbms/include/DB/DataStreams/IBlockInputStream.h +++ b/dbms/include/DB/DataStreams/IBlockInputStream.h @@ -18,7 +18,7 @@ using Poco::SharedPtr; * Функция принимает количество строк в последнем блоке, количество байт в последнем блоке. * Следует иметь ввиду, что колбэк может вызываться из разных потоков. */ -typedef boost::function ProgressCallback; +typedef std::function ProgressCallback; /** Интерфейс потока для чтения данных по блокам из БД. diff --git a/dbms/src/Client/Benchmark.cpp b/dbms/src/Client/Benchmark.cpp new file mode 100644 index 00000000000..b87fc8b008e --- /dev/null +++ b/dbms/src/Client/Benchmark.cpp @@ -0,0 +1,313 @@ +#define DBMS_CLIENT 1 /// Используется в Context.h + +#include +#include +#include +#include + +#include +#include +#include + +#include +#include +#include + +#include +#include +#include + +#include + +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +#include +#include + +#include + +#include + +#include "InterruptListener.h" + + +/** Инструмент для измерения производительности ClickHouse + * при выполнении запросов с фиксированным количеством одновременных запросов. + */ + +namespace DB +{ + +class Benchmark +{ +public: + Benchmark(unsigned concurrency_, + const String & host_, UInt16 port_, const String & default_database_, + const String & user_, const String & password_) + : concurrency(concurrency_), queue(concurrency), pool(concurrency), + connections(concurrency, host_, port_, default_database_, user_, password_, data_type_factory) + { + std::cerr << std::fixed << std::setprecision(3); + + readQueries(); + run(); + report(); + } + +private: + typedef std::string Query; + + unsigned concurrency; + + typedef std::vector Queries; + Queries queries; + + typedef ConcurrentBoundedQueue Queue; + Queue queue; + + boost::threadpool::pool pool; + + DataTypeFactory data_type_factory; + ConnectionPool connections; + + Stopwatch total_watch; + size_t total_queries = 0; + size_t total_rows = 0; + size_t total_bytes = 0; + ReservoirSampler sampler {1 << 16}; + Poco::FastMutex mutex; + + + void readQueries() + { + ReadBufferFromFileDescriptor in(STDIN_FILENO); + + while (!in.eof()) + { + std::string query; + readText(query, in); + assertString("\n", in); + + if (!query.empty()) + queries.emplace_back(query); + } + + if (queries.empty()) + throw Exception("Empty list of queries."); + + std::cerr << "Loaded " << queries.size() << " queries." << std::endl; + } + + + void run() + { + for (size_t i = 0; i < concurrency; ++i) + pool.schedule(std::bind(&Benchmark::thread, this, connections.get())); + + InterruptListener interrupt_listener; + + total_watch.restart(); + Stopwatch watch; + + /// В цикле, кладём все запросы в очередь. + for (size_t i; !interrupt_listener.check(); ++i) + { + if (i >= queries.size()) + i = 0; + + queue.push(queries[i]); + + if (watch.elapsedSeconds() > 1) + { + report(); + watch.restart(); + } + } + + /// Попросим потоки завершиться. + for (size_t i = 0; i < concurrency; ++i) + queue.push(""); + + pool.wait(); + } + + + void thread(ConnectionPool::Entry connection) + { + try + { + /// В этих потоках не будем принимать сигнал INT. + sigset_t sig_set; + if (sigemptyset(&sig_set) + || sigaddset(&sig_set, SIGINT) + || pthread_sigmask(SIG_BLOCK, &sig_set, NULL)) + throwFromErrno("Cannot block signal.", ErrorCodes::CANNOT_BLOCK_SIGNAL); + + Query query; + + while (true) + { + queue.pop(query); + + /// Пустой запрос обозначает конец работы. + if (query.empty()) + break; + + execute(connection, query); + } + } + catch (const Exception & e) + { + std::string text = e.displayText(); + + std::cerr << "Code: " << e.code() << ". " << text << std::endl << std::endl; + + /// Если есть стек-трейс на сервере, то не будем писать стек-трейс на клиенте. + if (std::string::npos == text.find("Stack trace")) + std::cerr << "Stack trace:" << std::endl + << e.getStackTrace().toString(); + + throw; + } + catch (const Poco::Exception & e) + { + std::cerr << "Poco::Exception: " << e.displayText() << std::endl; + throw; + } + catch (const std::exception & e) + { + std::cerr << "std::exception: " << e.what() << std::endl; + throw; + } + catch (...) + { + std::cerr << "Unknown exception" << std::endl; + throw; + } + } + + + void execute(ConnectionPool::Entry & connection, Query & query) + { + Stopwatch watch; + RemoteBlockInputStream stream(*connection, query, nullptr); + + size_t rows = 0; + size_t bytes = 0; + stream.setProgressCallback([&](size_t rows_inc , size_t bytes_inc) { rows += rows_inc; bytes += bytes_inc; }); + + stream.readPrefix(); + while (Block block = stream.read()) + ; + stream.readSuffix(); + + addTiming(watch.elapsedSeconds(), rows, bytes); + } + + + void addTiming(double seconds, size_t rows, size_t bytes) + { + Poco::ScopedLock lock(mutex); + + ++total_queries; + total_rows += rows; + total_bytes += bytes; + sampler.insert(seconds); + } + + + void report() + { + Poco::ScopedLock lock(mutex); + + std::cerr + << std::endl + << "QPS: " << (total_queries / total_watch.elapsedSeconds()) << ", " + << "RPS: " << (total_rows / total_watch.elapsedSeconds()) << ", " + << "MiB/s: " << (total_bytes / total_watch.elapsedSeconds() / 1048576) << "." + << std::endl; + + for (double level = 0; level < 1; level += 0.1) + std::cerr << int(level * 100) << "%\t" << sampler.quantileInterpolated(level) << " sec." << std::endl; + } +}; + +} + + +int main(int argc, char ** argv) +{ + using namespace DB; + + try + { + boost::program_options::options_description desc("Allowed options"); + desc.add_options() + ("help", "produce help message") + ("concurrency,c", boost::program_options::value()->default_value(1), "number of parallel queries") + ("host,h", boost::program_options::value()->default_value("localhost"), "") + ("port", boost::program_options::value()->default_value(9000), "") + ("user", boost::program_options::value()->default_value("default"), "") + ("password", boost::program_options::value()->default_value(""), "") + ("database", boost::program_options::value()->default_value("default"), "") + ; + + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " << argv[0] << " [options] < queries.txt" << std::endl; + std::cout << desc << std::endl; + return 1; + } + + Benchmark benchmark( + options["concurrency"].as(), + options["host"].as(), + options["port"].as(), + options["database"].as(), + options["user"].as(), + options["password"].as()); + } + catch (const Exception & e) + { + std::string text = e.displayText(); + + std::cerr << "Code: " << e.code() << ". " << text << std::endl << std::endl; + + /// Если есть стек-трейс на сервере, то не будем писать стек-трейс на клиенте. + if (std::string::npos == text.find("Stack trace")) + std::cerr << "Stack trace:" << std::endl + << e.getStackTrace().toString(); + + return e.code(); + } + catch (const Poco::Exception & e) + { + std::cerr << "Poco::Exception: " << e.displayText() << std::endl; + return ErrorCodes::POCO_EXCEPTION; + } + catch (const std::exception & e) + { + std::cerr << "std::exception: " << e.what() << std::endl; + return ErrorCodes::STD_EXCEPTION; + } + catch (...) + { + std::cerr << "Unknown exception" << std::endl; + return ErrorCodes::UNKNOWN_EXCEPTION; + } +} diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index f5e590abd11..3d542ac2b98 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -44,6 +44,8 @@ #include +#include "InterruptListener.h" + /** Клиент командной строки СУБД ClickHouse. */ @@ -55,75 +57,6 @@ namespace DB using Poco::SharedPtr; -/** Пока существует объект этого класса - блокирует сигнал INT, при этом позволяет узнать, не пришёл ли он. - * Это нужно, чтобы можно было прервать выполнение запроса с помощью Ctrl+C. - * В один момент времени используйте только один экземпляр этого класса. - * Если метод check вернул true (пришёл сигнал), то следующие вызовы будут ждать следующий сигнал. - */ -class InterruptListener -{ -private: - bool active; - sigset_t sig_set; - -public: - InterruptListener() : active(false) - { - if (sigemptyset(&sig_set) - || sigaddset(&sig_set, SIGINT)) - throwFromErrno("Cannot manipulate with signal set.", ErrorCodes::CANNOT_MANIPULATE_SIGSET); - - block(); - } - - ~InterruptListener() - { - unblock(); - } - - bool check() - { - if (!active) - return false; - - timespec timeout = { 0, 0 }; - - if (-1 == sigtimedwait(&sig_set, NULL, &timeout)) - { - if (errno == EAGAIN) - return false; - else - throwFromErrno("Cannot poll signal (sigtimedwait).", ErrorCodes::CANNOT_WAIT_FOR_SIGNAL); - } - - return true; - } - - void block() - { - if (!active) - { - if (pthread_sigmask(SIG_BLOCK, &sig_set, NULL)) - throwFromErrno("Cannot block signal.", ErrorCodes::CANNOT_BLOCK_SIGNAL); - - active = true; - } - } - - /// Можно прекратить блокировать сигнал раньше, чем в деструкторе. - void unblock() - { - if (active) - { - if (pthread_sigmask(SIG_UNBLOCK, &sig_set, NULL)) - throwFromErrno("Cannot unblock signal.", ErrorCodes::CANNOT_UNBLOCK_SIGNAL); - - active = false; - } - } -}; - - class Client : public Poco::Util::Application { public: diff --git a/dbms/src/Client/InterruptListener.h b/dbms/src/Client/InterruptListener.h new file mode 100644 index 00000000000..f586e4c6c55 --- /dev/null +++ b/dbms/src/Client/InterruptListener.h @@ -0,0 +1,79 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/** Пока существует объект этого класса - блокирует сигнал INT, при этом позволяет узнать, не пришёл ли он. + * Это нужно, чтобы можно было прервать выполнение запроса с помощью Ctrl+C. + * В один момент времени используйте только один экземпляр этого класса. + * Если метод check вернул true (пришёл сигнал), то следующие вызовы будут ждать следующий сигнал. + */ +class InterruptListener +{ +private: + bool active; + sigset_t sig_set; + +public: + InterruptListener() : active(false) + { + if (sigemptyset(&sig_set) + || sigaddset(&sig_set, SIGINT)) + throwFromErrno("Cannot manipulate with signal set.", ErrorCodes::CANNOT_MANIPULATE_SIGSET); + + block(); + } + + ~InterruptListener() + { + unblock(); + } + + bool check() + { + if (!active) + return false; + + timespec timeout = { 0, 0 }; + + if (-1 == sigtimedwait(&sig_set, NULL, &timeout)) + { + if (errno == EAGAIN) + return false; + else + throwFromErrno("Cannot poll signal (sigtimedwait).", ErrorCodes::CANNOT_WAIT_FOR_SIGNAL); + } + + return true; + } + + void block() + { + if (!active) + { + if (pthread_sigmask(SIG_BLOCK, &sig_set, NULL)) + throwFromErrno("Cannot block signal.", ErrorCodes::CANNOT_BLOCK_SIGNAL); + + active = true; + } + } + + /// Можно прекратить блокировать сигнал раньше, чем в деструкторе. + void unblock() + { + if (active) + { + if (pthread_sigmask(SIG_UNBLOCK, &sig_set, NULL)) + throwFromErrno("Cannot unblock signal.", ErrorCodes::CANNOT_UNBLOCK_SIGNAL); + + active = false; + } + } +}; + +} From 913ef0a7a3a2787a9036c0979ac4752c78dba1cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 13:41:57 +0400 Subject: [PATCH 04/23] Addition to prev. revision [#METR-2944]. --- dbms/src/Client/Benchmark.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Client/Benchmark.cpp b/dbms/src/Client/Benchmark.cpp index b87fc8b008e..21a36be3a80 100644 --- a/dbms/src/Client/Benchmark.cpp +++ b/dbms/src/Client/Benchmark.cpp @@ -122,7 +122,7 @@ private: Stopwatch watch; /// В цикле, кладём все запросы в очередь. - for (size_t i; !interrupt_listener.check(); ++i) + for (size_t i = 0; !interrupt_listener.check(); ++i) { if (i >= queries.size()) i = 0; From 88058d9971e100da92b004b431cd1e64fede97ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 16:25:37 +0400 Subject: [PATCH 05/23] dbms: improved performance of short queries [#METR-2944]. --- dbms/include/DB/Parsers/CommonParsers.h | 39 +++++----- .../DB/Parsers/ExpressionElementParsers.h | 56 +++++++------- .../DB/Parsers/ExpressionListParsers.h | 76 +++++++++---------- dbms/include/DB/Parsers/IParser.h | 15 ++-- dbms/include/DB/Parsers/IParserBase.h | 9 +-- dbms/include/DB/Parsers/ParserAlterQuery.h | 4 +- dbms/include/DB/Parsers/ParserCreateQuery.h | 28 +++---- dbms/include/DB/Parsers/ParserDropQuery.h | 4 +- dbms/include/DB/Parsers/ParserInsertQuery.h | 4 +- dbms/include/DB/Parsers/ParserOptimizeQuery.h | 4 +- dbms/include/DB/Parsers/ParserQuery.h | 4 +- dbms/include/DB/Parsers/ParserRenameQuery.h | 4 +- dbms/include/DB/Parsers/ParserSelectQuery.h | 4 +- dbms/include/DB/Parsers/ParserSetQuery.h | 4 +- .../DB/Parsers/ParserShowProcesslistQuery.h | 4 +- .../DB/Parsers/ParserShowTablesQuery.h | 4 +- .../DB/Parsers/ParserTablePropertiesQuery.h | 4 +- dbms/include/DB/Parsers/ParserUseQuery.h | 4 +- dbms/src/Client/Client.cpp | 2 +- .../DataStreams/tests/expression_stream.cpp | 2 +- dbms/src/DataStreams/tests/filter_stream.cpp | 2 +- .../tests/filter_stream_hitlog.cpp | 2 +- dbms/src/DataStreams/tests/fork_streams.cpp | 2 +- dbms/src/DataStreams/tests/sorting_stream.cpp | 2 +- dbms/src/DataTypes/DataTypeFactory.cpp | 6 +- dbms/src/Interpreters/Context.cpp | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../Interpreters/InterpreterRenameQuery.cpp | 2 +- dbms/src/Interpreters/executeQuery.cpp | 4 +- dbms/src/Interpreters/loadMetadata.cpp | 2 +- dbms/src/Interpreters/tests/create_query.cpp | 2 +- dbms/src/Interpreters/tests/expression.cpp | 2 +- .../tests/expression_analyzer.cpp | 2 +- dbms/src/Parsers/ExpressionElementParsers.cpp | 28 +++---- dbms/src/Parsers/ExpressionListParsers.cpp | 22 +++--- dbms/src/Parsers/ParserAlterQuery.cpp | 2 +- dbms/src/Parsers/ParserCreateQuery.cpp | 14 ++-- dbms/src/Parsers/ParserDropQuery.cpp | 2 +- dbms/src/Parsers/ParserInsertQuery.cpp | 2 +- dbms/src/Parsers/ParserOptimizeQuery.cpp | 2 +- dbms/src/Parsers/ParserQuery.cpp | 2 +- dbms/src/Parsers/ParserRenameQuery.cpp | 4 +- dbms/src/Parsers/ParserSelectQuery.cpp | 2 +- dbms/src/Parsers/ParserSetQuery.cpp | 4 +- dbms/src/Parsers/ParserShowTablesQuery.cpp | 2 +- .../Parsers/ParserTablePropertiesQuery.cpp | 2 +- dbms/src/Parsers/tests/create_parser.cpp | 2 +- dbms/src/Parsers/tests/select_parser.cpp | 2 +- 48 files changed, 197 insertions(+), 202 deletions(-) diff --git a/dbms/include/DB/Parsers/CommonParsers.h b/dbms/include/DB/Parsers/CommonParsers.h index 2f6ea5a1fd1..0d9b4d1f6e1 100644 --- a/dbms/include/DB/Parsers/CommonParsers.h +++ b/dbms/include/DB/Parsers/CommonParsers.h @@ -18,7 +18,8 @@ using Poco::SharedPtr; class ParserString : public IParserBase { private: - String s; + const char * s; + size_t s_size; bool word_boundary; bool case_insensitive; @@ -28,23 +29,23 @@ private: } public: - ParserString(const String & s_, bool word_boundary_ = false, bool case_insensitive_ = false) - : s(s_), word_boundary(word_boundary_), case_insensitive(case_insensitive_) {} + ParserString(const char * s_, bool word_boundary_ = false, bool case_insensitive_ = false) + : s(s_), s_size(strlen(s)), word_boundary(word_boundary_), case_insensitive(case_insensitive_) {} protected: - String getName() { return s; } + const char * getName() const { return s; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { - if (static_cast(s.size()) > end - pos || (case_insensitive ? strncasecmp : strncmp)(pos, s.data(), s.size())) + if (static_cast(s_size) > end - pos || (case_insensitive ? strncasecmp : strncmp)(pos, s, s_size)) return false; else { - if (word_boundary && s.size() && is_word(*s.rbegin()) - && pos + s.size() != end && is_word(pos[s.size()])) + if (word_boundary && s_size && is_word(s[s_size - 1]) + && pos + s_size != end && is_word(pos[s_size])) return false; - pos += s.size(); + pos += s_size; return true; } } @@ -61,9 +62,9 @@ public: protected: bool allow_newlines; - String getName() { return "white space"; } + const char * getName() const { return "white space"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; while (*pos == ' ' || *pos == '\t' || (allow_newlines && *pos == '\n') || *pos == '\r' || *pos == '\f') @@ -77,9 +78,9 @@ protected: class ParserCStyleComment : public IParserBase { protected: - String getName() { return "C-style comment"; } + const char * getName() const { return "C-style comment"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { if (end - pos >= 4 && pos[0] == '/' && pos[1] == '*') { @@ -107,9 +108,9 @@ protected: class ParserSQLStyleComment : public IParserBase { protected: - String getName() { return "SQL-style comment"; } + const char * getName() const { return "SQL-style comment"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { if (end - pos >= 2 && pos[0] == '-' && pos[1] == '-') { @@ -132,9 +133,9 @@ protected: class ParserComment : public IParserBase { protected: - String getName() { return "comment"; } + const char * getName() const { return "comment"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserCStyleComment p1; ParserSQLStyleComment p2; @@ -153,9 +154,9 @@ public: protected: bool allow_newlines_outside_comments; - String getName() { return "white space or comments"; } + const char * getName() const { return "white space or comments"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserWhiteSpace p1(allow_newlines_outside_comments); ParserComment p2; diff --git a/dbms/include/DB/Parsers/ExpressionElementParsers.h b/dbms/include/DB/Parsers/ExpressionElementParsers.h index 74969276955..1e57e591db9 100644 --- a/dbms/include/DB/Parsers/ExpressionElementParsers.h +++ b/dbms/include/DB/Parsers/ExpressionElementParsers.h @@ -10,8 +10,8 @@ namespace DB class ParserArray : public IParserBase { protected: - String getName() { return "array"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "array"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -22,8 +22,8 @@ protected: class ParserParenthesisExpression : public IParserBase { protected: - String getName() { return "expression in parenthesis"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "expression in parenthesis"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -32,8 +32,8 @@ protected: class ParserSubquery : public IParserBase { protected: - String getName() { return "SELECT subquery"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "SELECT subquery"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -42,8 +42,8 @@ protected: class ParserIdentifier : public IParserBase { protected: - String getName() { return "identifier"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "identifier"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -52,8 +52,8 @@ protected: class ParserCompoundIdentifier : public IParserBase { protected: - String getName() { return "compound identifier"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "compound identifier"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -65,8 +65,8 @@ protected: class ParserFunction : public IParserBase { protected: - String getName() { return "function"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "function"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -75,8 +75,8 @@ protected: class ParserNull : public IParserBase { protected: - String getName() { return "NULL"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "NULL"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -85,8 +85,8 @@ protected: class ParserNumber : public IParserBase { protected: - String getName() { return "number"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "number"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -95,8 +95,8 @@ protected: class ParserStringLiteral : public IParserBase { protected: - String getName() { return "string literal"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "string literal"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -105,8 +105,8 @@ protected: class ParserLiteral : public IParserBase { protected: - String getName() { return "literal"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "literal"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -115,8 +115,8 @@ protected: class ParserAlias : public IParserBase { protected: - String getName() { return "alias"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "alias"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -125,8 +125,8 @@ protected: class ParserExpressionElement : public IParserBase { protected: - String getName() { return "element of expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "element of expression"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -139,8 +139,8 @@ public: protected: ParserPtr elem_parser; - String getName() { return "element of expression with optional alias"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "element of expression with optional alias"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -150,8 +150,8 @@ protected: class ParserOrderByElement : public IParserBase { protected: - String getName() { return "element of ORDER BY expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "element of ORDER BY expression"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; diff --git a/dbms/include/DB/Parsers/ExpressionListParsers.h b/dbms/include/DB/Parsers/ExpressionListParsers.h index 117feb31e0b..b99f93b675c 100644 --- a/dbms/include/DB/Parsers/ExpressionListParsers.h +++ b/dbms/include/DB/Parsers/ExpressionListParsers.h @@ -14,7 +14,7 @@ namespace DB /** Оператор и соответствующая ему функция. Например, "+" -> "plus" * Не std::map, так как порядок парсинга операторов задаётся явно и может отличаться от алфавитного. */ -typedef std::list > Operators_t; +typedef std::list > Operators_t; /** Список элементов, разделённых чем-либо. */ @@ -26,8 +26,8 @@ public: { } protected: - String getName() { return "list of elements (" + elem_parser->getName() + ")"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "list of elements"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); private: ParserPtr elem_parser; ParserPtr separator_parser; @@ -53,9 +53,9 @@ public: } protected: - String getName() { return "list, delimited by binary operators"; } + const char * getName() const { return "list, delimited by binary operators"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -70,15 +70,15 @@ private: ParserPtr elem_parser; public: - ParserVariableArityOperatorList(const String & infix_, const String & function_, ParserPtr elem_parser_) + ParserVariableArityOperatorList(const char * infix_, const String & function_, ParserPtr elem_parser_) : infix_parser(infix_, true, true), function_name(function_), elem_parser(elem_parser_) { } protected: - String getName() { return "list, delimited by operator of variable arity"; } + const char * getName() const { return "list, delimited by operator of variable arity"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -100,8 +100,8 @@ public: } protected: - String getName() { return "expression with prefix unary operator"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "expression with prefix unary operator"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -114,9 +114,9 @@ public: ParserAccessExpression(); protected: - String getName() { return "access expression"; } + const char * getName() const { return "access expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return operator_parser.parse(pos, end, node, expected); } @@ -136,9 +136,9 @@ public: } protected: - String getName() { return "unary minus expression"; } + const char * getName() const { return "unary minus expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -159,9 +159,9 @@ public: } protected: - String getName() { return "multiplicative expression"; } + const char * getName() const { return "multiplicative expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return operator_parser.parse(pos, end, node, expected); } @@ -184,9 +184,9 @@ public: } protected: - String getName() { return "additive expression"; } + const char * getName() const { return "additive expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return operator_parser.parse(pos, end, node, expected); } @@ -219,9 +219,9 @@ public: } protected: - String getName() { return "comparison expression"; } + const char * getName() const { return "comparison expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return operator_parser.parse(pos, end, node, expected); } @@ -241,9 +241,9 @@ public: } protected: - String getName() { return "logical-NOT expression"; } + const char * getName() const { return "logical-NOT expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return operator_parser.parse(pos, end, node, expected); } @@ -263,9 +263,9 @@ public: } protected: - String getName() { return "logical-AND expression"; } + const char * getName() const { return "logical-AND expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return operator_parser.parse(pos, end, node, expected); } @@ -285,9 +285,9 @@ public: } protected: - String getName() { return "logical-OR expression"; } + const char * getName() const { return "logical-OR expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return operator_parser.parse(pos, end, node, expected); } @@ -309,9 +309,9 @@ public: } protected: - String getName() { return "expression with ternary operator"; } + const char * getName() const { return "expression with ternary operator"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -327,9 +327,9 @@ public: } protected: - String getName() { return "lambda expression"; } + const char * getName() const { return "lambda expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -340,9 +340,9 @@ public: protected: ParserPtr impl; - String getName() { return "expression with optional alias"; } + const char * getName() const { return "expression with optional alias"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return impl->parse(pos, end, node, expected); } @@ -353,8 +353,8 @@ protected: class ParserExpressionList : public IParserBase { protected: - String getName() { return "list of expressions"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "list of expressions"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -363,16 +363,16 @@ class ParserNotEmptyExpressionList : public IParserBase private: ParserExpressionList nested_parser; protected: - String getName() { return "not empty list of expressions"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "not empty list of expressions"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; class ParserOrderByExpressionList : public IParserBase { protected: - String getName() { return "order by expression"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "order by expression"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; diff --git a/dbms/include/DB/Parsers/IParser.h b/dbms/include/DB/Parsers/IParser.h index b689824c9f0..52d9bc88ec7 100644 --- a/dbms/include/DB/Parsers/IParser.h +++ b/dbms/include/DB/Parsers/IParser.h @@ -1,5 +1,4 @@ -#ifndef DBMS_PARSERS_IPARSER_H -#define DBMS_PARSERS_IPARSER_H +#pragma once #include #include @@ -22,7 +21,7 @@ public: typedef const char * Pos; /** Получить текст о том, что парсит этот парсер. */ - virtual String getName() = 0; + virtual const char * getName() const = 0; /** Распарсить кусок текста с позиции pos, но не дальше конца строки (end - позиция после конца строки), * переместить указатель pos на максимальное место, до которого удалось распарсить, @@ -32,9 +31,9 @@ public: * или что парсит этот парсер, если парсинг был успешным. * Везде предполагается, что строка, в которую входит диапазон [begin, end) 0-terminated. */ - virtual bool parse(Pos & pos, Pos end, ASTPtr & node, String & expected) = 0; + virtual bool parse(Pos & pos, Pos end, ASTPtr & node, const char *& expected) = 0; - bool ignore(Pos & pos, Pos end, String & expected) + bool ignore(Pos & pos, Pos end, const char *& expected) { ASTPtr ignore_node; return parse(pos, end, ignore_node, expected); @@ -42,13 +41,13 @@ public: bool ignore(Pos & pos, Pos end) { - String expected; + const char * expected; return ignore(pos, end, expected); } /** То же самое, но не двигать позицию и не записывать результат в node. */ - bool check(Pos & pos, Pos end, String & expected) + bool check(Pos & pos, Pos end, const char *& expected) { Pos begin = pos; ASTPtr node; @@ -67,5 +66,3 @@ public: typedef SharedPtr ParserPtr; } - -#endif diff --git a/dbms/include/DB/Parsers/IParserBase.h b/dbms/include/DB/Parsers/IParserBase.h index acb996e45bf..aa777fcca50 100644 --- a/dbms/include/DB/Parsers/IParserBase.h +++ b/dbms/include/DB/Parsers/IParserBase.h @@ -1,5 +1,4 @@ -#ifndef DBMS_PARSERS_IPARSERBASE_H -#define DBMS_PARSERS_IPARSERBASE_H +#pragma once #include #include @@ -17,16 +16,14 @@ namespace DB class IParserBase : public IParser { public: - bool parse(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parse(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { expected = getName(); bool res = parseImpl(pos, end, node, expected); return res; } protected: - virtual bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) = 0; + virtual bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) = 0; }; } - -#endif diff --git a/dbms/include/DB/Parsers/ParserAlterQuery.h b/dbms/include/DB/Parsers/ParserAlterQuery.h index 3d132e0431a..872f103a4b5 100644 --- a/dbms/include/DB/Parsers/ParserAlterQuery.h +++ b/dbms/include/DB/Parsers/ParserAlterQuery.h @@ -13,8 +13,8 @@ namespace DB class ParserAlterQuery : public IParserBase { protected: - String getName() { return "ALTER query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "ALTER query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserCreateQuery.h b/dbms/include/DB/Parsers/ParserCreateQuery.h index 82e793a539c..b84a8add620 100644 --- a/dbms/include/DB/Parsers/ParserCreateQuery.h +++ b/dbms/include/DB/Parsers/ParserCreateQuery.h @@ -12,8 +12,8 @@ namespace DB class ParserNestedTable : public IParserBase { protected: - String getName() { return "nested table"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "nested table"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -26,8 +26,8 @@ protected: class ParserIdentifierWithParameters : public IParserBase { protected: - String getName() { return "identifier with parameters"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "identifier with parameters"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -37,8 +37,8 @@ protected: class ParserIdentifierWithOptionalParameters : public IParserBase { protected: - String getName() { return "identifier with optional parameters"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "identifier with optional parameters"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -46,8 +46,8 @@ protected: class ParserNameTypePair : public IParserBase { protected: - String getName() { return "name and type pair"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "name and type pair"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -55,8 +55,8 @@ protected: class ParserNameTypePairList : public IParserBase { protected: - String getName() { return "name and type pair list"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "name and type pair list"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -64,8 +64,8 @@ protected: class ParserEngine : public IParserBase { protected: - String getName() { return "ENGINE"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "ENGINE"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; @@ -89,8 +89,8 @@ protected: class ParserCreateQuery : public IParserBase { protected: - String getName() { return "CREATE TABLE or ATTACH TABLE query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "CREATE TABLE or ATTACH TABLE query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserDropQuery.h b/dbms/include/DB/Parsers/ParserDropQuery.h index 228e1d1bf78..4fc61d7f431 100644 --- a/dbms/include/DB/Parsers/ParserDropQuery.h +++ b/dbms/include/DB/Parsers/ParserDropQuery.h @@ -16,8 +16,8 @@ namespace DB class ParserDropQuery : public IParserBase { protected: - String getName() { return "DROP query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "DROP query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserInsertQuery.h b/dbms/include/DB/Parsers/ParserInsertQuery.h index 68bab4a8923..c8e27f65850 100644 --- a/dbms/include/DB/Parsers/ParserInsertQuery.h +++ b/dbms/include/DB/Parsers/ParserInsertQuery.h @@ -25,8 +25,8 @@ namespace DB class ParserInsertQuery : public IParserBase { protected: - String getName() { return "INSERT query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "INSERT query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserOptimizeQuery.h b/dbms/include/DB/Parsers/ParserOptimizeQuery.h index 669a122c043..dc716a40e97 100644 --- a/dbms/include/DB/Parsers/ParserOptimizeQuery.h +++ b/dbms/include/DB/Parsers/ParserOptimizeQuery.h @@ -12,8 +12,8 @@ namespace DB class ParserOptimizeQuery : public IParserBase { protected: - String getName() { return "OPTIMIZE query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "OPTIMIZE query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserQuery.h b/dbms/include/DB/Parsers/ParserQuery.h index 622093db792..49a39e48ca1 100644 --- a/dbms/include/DB/Parsers/ParserQuery.h +++ b/dbms/include/DB/Parsers/ParserQuery.h @@ -10,8 +10,8 @@ namespace DB class ParserQuery : public IParserBase { protected: - String getName() { return "Query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "Query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserRenameQuery.h b/dbms/include/DB/Parsers/ParserRenameQuery.h index fd4d71b2c3c..90091111901 100644 --- a/dbms/include/DB/Parsers/ParserRenameQuery.h +++ b/dbms/include/DB/Parsers/ParserRenameQuery.h @@ -14,8 +14,8 @@ namespace DB class ParserRenameQuery : public IParserBase { protected: - String getName() { return "RENAME query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "RENAME query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserSelectQuery.h b/dbms/include/DB/Parsers/ParserSelectQuery.h index 38082d4701a..66b0711836e 100644 --- a/dbms/include/DB/Parsers/ParserSelectQuery.h +++ b/dbms/include/DB/Parsers/ParserSelectQuery.h @@ -10,8 +10,8 @@ namespace DB class ParserSelectQuery : public IParserBase { protected: - String getName() { return "SELECT query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "SELECT query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserSetQuery.h b/dbms/include/DB/Parsers/ParserSetQuery.h index e06e814bd95..93a39b26961 100644 --- a/dbms/include/DB/Parsers/ParserSetQuery.h +++ b/dbms/include/DB/Parsers/ParserSetQuery.h @@ -13,8 +13,8 @@ namespace DB class ParserSetQuery : public IParserBase { protected: - String getName() { return "SET query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "SET query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h b/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h index 24290567313..093fba9cb9d 100644 --- a/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h +++ b/dbms/include/DB/Parsers/ParserShowProcesslistQuery.h @@ -15,9 +15,9 @@ namespace DB class ParserShowProcesslistQuery : public IParserBase { protected: - String getName() { return "SHOW PROCESSLIST query"; } + const char * getName() const { return "SHOW PROCESSLIST query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/include/DB/Parsers/ParserShowTablesQuery.h b/dbms/include/DB/Parsers/ParserShowTablesQuery.h index 91df05ab500..82bb0bfa2bf 100644 --- a/dbms/include/DB/Parsers/ParserShowTablesQuery.h +++ b/dbms/include/DB/Parsers/ParserShowTablesQuery.h @@ -14,8 +14,8 @@ namespace DB class ParserShowTablesQuery : public IParserBase { protected: - String getName() { return "SHOW TABLES|DATABASES query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "SHOW TABLES|DATABASES query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h b/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h index b5faf6f5049..64d20b5ff00 100644 --- a/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h +++ b/dbms/include/DB/Parsers/ParserTablePropertiesQuery.h @@ -12,8 +12,8 @@ namespace DB class ParserTablePropertiesQuery : public IParserBase { protected: - String getName() { return "EXISTS, SHOW CREATE or DESCRIBE query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected); + const char * getName() const { return "EXISTS, SHOW CREATE or DESCRIBE query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected); }; } diff --git a/dbms/include/DB/Parsers/ParserUseQuery.h b/dbms/include/DB/Parsers/ParserUseQuery.h index 639798b0050..309c693f4cc 100644 --- a/dbms/include/DB/Parsers/ParserUseQuery.h +++ b/dbms/include/DB/Parsers/ParserUseQuery.h @@ -15,8 +15,8 @@ namespace DB class ParserUseQuery : public IParserBase { protected: - String getName() { return "USE query"; } - bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) + const char * getName() const { return "USE query"; } + bool parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 3d542ac2b98..9f42db41403 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -461,7 +461,7 @@ private: bool parseQuery() { ParserQuery parser; - std::string expected; + const char * expected = ""; const char * begin = query.data(); const char * end = begin + query.size(); diff --git a/dbms/src/DataStreams/tests/expression_stream.cpp b/dbms/src/DataStreams/tests/expression_stream.cpp index 6aa4771f105..f38c19e47a2 100644 --- a/dbms/src/DataStreams/tests/expression_stream.cpp +++ b/dbms/src/DataStreams/tests/expression_stream.cpp @@ -32,7 +32,7 @@ int main(int argc, char ** argv) DB::ParserSelectQuery parser; DB::ASTPtr ast; std::string input = "SELECT number, number / 3, number * number"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); diff --git a/dbms/src/DataStreams/tests/filter_stream.cpp b/dbms/src/DataStreams/tests/filter_stream.cpp index df38896dc5e..e6d121c27ea 100644 --- a/dbms/src/DataStreams/tests/filter_stream.cpp +++ b/dbms/src/DataStreams/tests/filter_stream.cpp @@ -34,7 +34,7 @@ int main(int argc, char ** argv) DB::ParserSelectQuery parser; DB::ASTPtr ast; std::string input = "SELECT number, number % 3 == 1"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); diff --git a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp b/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp index 1a1b9b974ed..3502b798470 100644 --- a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp +++ b/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp @@ -105,7 +105,7 @@ int main(int argc, char ** argv) DB::ParserSelectQuery parser; DB::ASTPtr ast; std::string input = "SELECT UniqID, URL, CounterID, IsLink WHERE URL = 'http://mail.yandex.ru/neo2/#inbox'"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); diff --git a/dbms/src/DataStreams/tests/fork_streams.cpp b/dbms/src/DataStreams/tests/fork_streams.cpp index 51b5102e817..1a806b7b0e4 100644 --- a/dbms/src/DataStreams/tests/fork_streams.cpp +++ b/dbms/src/DataStreams/tests/fork_streams.cpp @@ -51,7 +51,7 @@ int main(int argc, char ** argv) DB::ParserSelectQuery parser; DB::ASTPtr ast; std::string input = "SELECT number, number % 10000000 == 1"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); diff --git a/dbms/src/DataStreams/tests/sorting_stream.cpp b/dbms/src/DataStreams/tests/sorting_stream.cpp index 8d25c866f76..f7f2a90eda9 100644 --- a/dbms/src/DataStreams/tests/sorting_stream.cpp +++ b/dbms/src/DataStreams/tests/sorting_stream.cpp @@ -107,7 +107,7 @@ int main(int argc, char ** argv) DB::ParserSelectQuery parser; DB::ASTPtr ast; std::string input = "SELECT UniqID, URL, CounterID, IsLink"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index bef4e5c7196..b5b812426f5 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -74,7 +74,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const ParserExpressionList args_parser; ASTPtr args_ast; - String expected; + const char * expected = ""; IParser::Pos pos = parameters.data(); IParser::Pos end = pos + parameters.size(); @@ -101,7 +101,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const { ParserNameTypePairList columns_p; ASTPtr columns_ast; - String expected; + const char * expected = ""; IParser::Pos pos = parameters.data(); IParser::Pos end = pos + parameters.size(); @@ -130,7 +130,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const { ParserExpressionList columns_p; ASTPtr columns_ast; - String expected; + const char * expected = ""; IParser::Pos pos = parameters.data(); IParser::Pos end = pos + parameters.size(); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 167bf366ae4..b4b1b30888b 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -284,7 +284,7 @@ ASTPtr Context::getCreateQuery(const String & database_name, const String & tabl ParserCreateQuery parser; ASTPtr ast; - String expected; + const char * expected = ""; bool parse_res = parser.parse(pos, end, ast, expected); /// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой. diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 3d745e9dfb5..60e01f5fd5c 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -137,7 +137,7 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists) StringPtr type_name = new String(it->second->getName()); ParserIdentifierWithOptionalParameters storage_p; - String expected; + const char * expected = ""; const char * pos = type_name->data(); const char * end = pos + type_name->size(); diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index dbd96738d3d..246d1c3d17c 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -72,7 +72,7 @@ void InterpreterRenameQuery::execute() const char * pos = create_query.data(); const char * end = pos + create_query.size(); ASTPtr ast; - String expected; + const char * expected = ""; bool parse_res = parser.parse(pos, end, ast, expected); /// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой. diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 5c2fc5fe2ee..a21eba175bb 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -31,7 +31,7 @@ void executeQuery( ParserQuery parser; ASTPtr ast; - std::string expected; + const char * expected = ""; std::vector parse_buf; const char * begin; @@ -122,7 +122,7 @@ BlockIO executeQuery( ParserQuery parser; ASTPtr ast; - std::string expected; + const char * expected = ""; const char * begin = query.data(); const char * end = begin + query.size(); diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 38685d71923..97812b819f8 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -28,7 +28,7 @@ static void executeCreateQuery(const String & query, Context & context, const St ParserCreateQuery parser; ASTPtr ast; - String expected; + const char * expected = ""; bool parse_res = parser.parse(pos, end, ast, expected); /// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой. diff --git a/dbms/src/Interpreters/tests/create_query.cpp b/dbms/src/Interpreters/tests/create_query.cpp index 9424ebbfb76..9aecbeecffe 100644 --- a/dbms/src/Interpreters/tests/create_query.cpp +++ b/dbms/src/Interpreters/tests/create_query.cpp @@ -73,7 +73,7 @@ int main(int argc, char ** argv) "DontCountHits UInt8,\n" "WithHash UInt8\n" ") ENGINE = Log"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); diff --git a/dbms/src/Interpreters/tests/expression.cpp b/dbms/src/Interpreters/tests/expression.cpp index 61e9f9e96ee..4cbb0d51e13 100644 --- a/dbms/src/Interpreters/tests/expression.cpp +++ b/dbms/src/Interpreters/tests/expression.cpp @@ -37,7 +37,7 @@ int main(int argc, char ** argv) "s1 <= 'abc', s1 <= s2, s1 >= 'abc', s1 >= s2, " "*/" "s1 < s2 AND x % 3 < x % 5"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index 5ee526d73e0..f4e67ca95cc 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -35,7 +35,7 @@ int main(int argc, char ** argv) IParser & parser = *parsers[i]; const char * pos = argv[1]; const char * end = argv[1] + strlen(argv[1]); - std::string expected; + const char * expected = ""; if (parser.parse(pos ,end, root, expected)) break; else diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 6adebca785d..92ae0fba3dc 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -24,7 +24,7 @@ namespace DB { -bool ParserArray::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserArray::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; ASTPtr contents_node; @@ -52,7 +52,7 @@ bool ParserArray::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected } -bool ParserParenthesisExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserParenthesisExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; ASTPtr contents_node; @@ -96,7 +96,7 @@ bool ParserParenthesisExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, S } -bool ParserSubquery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserSubquery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; ASTPtr select_node; @@ -120,7 +120,7 @@ bool ParserSubquery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expec } -bool ParserIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; @@ -154,7 +154,7 @@ bool ParserIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & exp } -bool ParserCompoundIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserCompoundIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; @@ -199,7 +199,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, Stri } -bool ParserFunction::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserFunction::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; @@ -258,7 +258,7 @@ bool ParserFunction::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expec } -bool ParserNull::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserNull::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; ParserString nested_parser("NULL", true); @@ -272,7 +272,7 @@ bool ParserNull::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) } -bool ParserNumber::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserNumber::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Field res; @@ -312,7 +312,7 @@ bool ParserNumber::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expecte } -bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; String s; @@ -360,7 +360,7 @@ bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & } -bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; @@ -385,7 +385,7 @@ bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expect } -bool ParserAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserWhiteSpaceOrComments ws; ParserString s_as("AS", true, true); @@ -403,7 +403,7 @@ bool ParserAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected } -bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; @@ -451,7 +451,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Strin } -bool ParserWithOptionalAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserWithOptionalAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserWhiteSpaceOrComments ws; ParserAlias alias_p; @@ -483,7 +483,7 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, Pos end, ASTPtr & node, Strin } -bool ParserOrderByElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserOrderByElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index b31152c5e41..f03b9c18759 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -12,7 +12,7 @@ namespace DB { -bool ParserList::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { bool first = true; ParserWhiteSpaceOrComments ws; @@ -54,7 +54,7 @@ bool ParserList::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) } -bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { bool first = true; ParserWhiteSpaceOrComments ws; @@ -117,7 +117,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP /** специальное исключение для оператора доступа к элементу массива x[y], который * содержит инфиксную часть '[' и суффиксную ']' (задаётся в виде '[') */ - if (it->first == "[") + if (0 == strcmp(it->first, "[")) { ParserString rest_p("]"); @@ -135,7 +135,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP return true; } -bool ParserVariableArityOperatorList::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserVariableArityOperatorList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserWhiteSpaceOrComments ws; @@ -179,7 +179,7 @@ bool ParserVariableArityOperatorList::parseImpl(Pos & pos, Pos end, ASTPtr & nod return true; } -bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserWhiteSpaceOrComments ws; ParserString symbol1("?"); @@ -244,7 +244,7 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & nod } -bool ParserLambdaExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserLambdaExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserWhiteSpaceOrComments ws; ParserString arrow("->"); @@ -314,7 +314,7 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, String } -bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserWhiteSpaceOrComments ws; @@ -365,7 +365,7 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & } -bool ParserUnaryMinusExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserUnaryMinusExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { /// В качестве исключения, отрицательные числа должны парситься, как литералы, а не как применение оператора. @@ -400,20 +400,20 @@ ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias() } -bool ParserExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return ParserList(new ParserExpressionWithOptionalAlias, new ParserString(",")).parse(pos, end, node, expected); } -bool ParserNotEmptyExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserNotEmptyExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return nested_parser.parse(pos, end, node, expected) && !dynamic_cast(*node).children.empty(); } -bool ParserOrderByExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserOrderByExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return ParserList(new ParserOrderByElement, new ParserString(","), false).parse(pos, end, node, expected); } diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index f2f001d4274..dc445d088f7 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -10,7 +10,7 @@ namespace DB { -bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 47c91727bcb..5ecbff94a0c 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -13,7 +13,7 @@ namespace DB { -bool ParserNestedTable::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserNestedTable::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserWhiteSpaceOrComments ws; ParserString open("("); @@ -55,7 +55,7 @@ bool ParserNestedTable::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex } -bool ParserIdentifierWithParameters::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserIdentifierWithParameters::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; @@ -75,7 +75,7 @@ bool ParserIdentifierWithParameters::parseImpl(Pos & pos, Pos end, ASTPtr & node } -bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserIdentifier non_parametric; ParserIdentifierWithParameters parametric; @@ -102,7 +102,7 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, Pos end, ASTPt } -bool ParserNameTypePair::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserNameTypePair::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserIdentifier name_parser; ParserIdentifierWithOptionalParameters type_parser; @@ -128,13 +128,13 @@ bool ParserNameTypePair::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & e } -bool ParserNameTypePairList::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserNameTypePairList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { return ParserList(new ParserNameTypePair, new ParserString(","), false).parse(pos, end, node, expected); } -bool ParserEngine::parseImpl(Pos & pos, Pos end, ASTPtr & storage, String & expected) +bool ParserEngine::parseImpl(Pos & pos, Pos end, ASTPtr & storage, const char *& expected) { ParserWhiteSpaceOrComments ws; ParserString s_engine("ENGINE", true, true); @@ -162,7 +162,7 @@ bool ParserEngine::parseImpl(Pos & pos, Pos end, ASTPtr & storage, String & expe } -bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserDropQuery.cpp b/dbms/src/Parsers/ParserDropQuery.cpp index 1ecef71cc0c..c6156e34854 100644 --- a/dbms/src/Parsers/ParserDropQuery.cpp +++ b/dbms/src/Parsers/ParserDropQuery.cpp @@ -9,7 +9,7 @@ namespace DB { -bool ParserDropQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserDropQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserInsertQuery.cpp b/dbms/src/Parsers/ParserInsertQuery.cpp index 4575b9a9520..88cc72f30a6 100644 --- a/dbms/src/Parsers/ParserInsertQuery.cpp +++ b/dbms/src/Parsers/ParserInsertQuery.cpp @@ -13,7 +13,7 @@ namespace DB { -bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserOptimizeQuery.cpp b/dbms/src/Parsers/ParserOptimizeQuery.cpp index 957f8de0208..a1a0065a78c 100644 --- a/dbms/src/Parsers/ParserOptimizeQuery.cpp +++ b/dbms/src/Parsers/ParserOptimizeQuery.cpp @@ -9,7 +9,7 @@ namespace DB { -bool ParserOptimizeQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserOptimizeQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserQuery.cpp b/dbms/src/Parsers/ParserQuery.cpp index b05d5dd577b..93d9124e2dd 100644 --- a/dbms/src/Parsers/ParserQuery.cpp +++ b/dbms/src/Parsers/ParserQuery.cpp @@ -18,7 +18,7 @@ namespace DB { -bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { ParserShowTablesQuery show_tables_p; ParserSelectQuery select_p; diff --git a/dbms/src/Parsers/ParserRenameQuery.cpp b/dbms/src/Parsers/ParserRenameQuery.cpp index b870e115777..374f3bd2bce 100644 --- a/dbms/src/Parsers/ParserRenameQuery.cpp +++ b/dbms/src/Parsers/ParserRenameQuery.cpp @@ -10,7 +10,7 @@ namespace DB /// Парсит database.table или table. -static bool parseDatabaseAndTable(ASTRenameQuery::Table & db_and_table, IParser::Pos & pos, IParser::Pos end, String & expected) +static bool parseDatabaseAndTable(ASTRenameQuery::Table & db_and_table, IParser::Pos & pos, IParser::Pos end, const char *& expected) { ParserIdentifier name_p; ParserWhiteSpaceOrComments ws; @@ -42,7 +42,7 @@ static bool parseDatabaseAndTable(ASTRenameQuery::Table & db_and_table, IParser: } -bool ParserRenameQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserRenameQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 1a2e81c7c8f..137b3a85721 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -12,7 +12,7 @@ namespace DB { -bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserSetQuery.cpp b/dbms/src/Parsers/ParserSetQuery.cpp index f4a340627b5..c3614d1da3b 100644 --- a/dbms/src/Parsers/ParserSetQuery.cpp +++ b/dbms/src/Parsers/ParserSetQuery.cpp @@ -11,7 +11,7 @@ namespace DB /// Парсит name = value. -static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos, IParser::Pos end, String & expected) +static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos, IParser::Pos end, const char *& expected) { ParserIdentifier name_p; ParserLiteral value_p; @@ -45,7 +45,7 @@ static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos, } -bool ParserSetQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserSetQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserShowTablesQuery.cpp b/dbms/src/Parsers/ParserShowTablesQuery.cpp index 0d110106cc1..2b17a404833 100644 --- a/dbms/src/Parsers/ParserShowTablesQuery.cpp +++ b/dbms/src/Parsers/ParserShowTablesQuery.cpp @@ -11,7 +11,7 @@ namespace DB { -bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/ParserTablePropertiesQuery.cpp b/dbms/src/Parsers/ParserTablePropertiesQuery.cpp index b5d6700934c..7bb7777e5e6 100644 --- a/dbms/src/Parsers/ParserTablePropertiesQuery.cpp +++ b/dbms/src/Parsers/ParserTablePropertiesQuery.cpp @@ -9,7 +9,7 @@ namespace DB { -bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected) +bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { Pos begin = pos; diff --git a/dbms/src/Parsers/tests/create_parser.cpp b/dbms/src/Parsers/tests/create_parser.cpp index 4f4ec983679..a7021a54650 100644 --- a/dbms/src/Parsers/tests/create_parser.cpp +++ b/dbms/src/Parsers/tests/create_parser.cpp @@ -12,7 +12,7 @@ int main(int argc, char ** argv) DB::ParserCreateQuery parser; DB::ASTPtr ast; std::string input = "CREATE TABLE hits (URL String, UserAgentMinor2 FixedString(2), EventTime DateTime) ENGINE = Log"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); diff --git a/dbms/src/Parsers/tests/select_parser.cpp b/dbms/src/Parsers/tests/select_parser.cpp index 86ec8f11556..d77cdea0850 100644 --- a/dbms/src/Parsers/tests/select_parser.cpp +++ b/dbms/src/Parsers/tests/select_parser.cpp @@ -19,7 +19,7 @@ int main(int argc, char ** argv) " HAVING SUM(Refresh) > 100" " ORDER BY Visits, PageViews" " LIMIT 1000, 10"; - std::string expected; + const char * expected = ""; const char * begin = input.data(); const char * end = begin + input.size(); From a75070d9c685c00b8ed1fba54d906c18c37e67b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 16:43:22 +0400 Subject: [PATCH 06/23] Addition to prev. revision [#METR-2944]. --- dbms/src/Storages/tests/merge_tree.cpp | 8 ++++---- dbms/src/Storages/tests/pk_condition.cpp | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/tests/merge_tree.cpp b/dbms/src/Storages/tests/merge_tree.cpp index 41c1ae9b524..a1e2de8e69f 100644 --- a/dbms/src/Storages/tests/merge_tree.cpp +++ b/dbms/src/Storages/tests/merge_tree.cpp @@ -33,12 +33,12 @@ int main(int argc, char ** argv) names_and_types->push_back(NameAndTypePair("a", new DataTypeArray(new DataTypeUInt32))); ASTPtr primary_expr; - String exprected; + const char * expected = ""; String primary_expr_str = "d"; const char * begin = primary_expr_str.data(); const char * end = begin + primary_expr_str.size(); ParserExpressionList parser; - if (!parser.parse(begin, end, primary_expr, exprected)) + if (!parser.parse(begin, end, primary_expr, expected)) throw Poco::Exception("Cannot parse " + primary_expr_str); StoragePtr table = StorageMergeTree::create("./", "test", names_and_types, context, primary_expr, "d", NULL, 101); @@ -82,12 +82,12 @@ int main(int argc, char ** argv) QueryProcessingStage::Enum stage; ASTPtr select; - String exprected; + const char * expected = ""; String select_str = "SELECT * FROM test"; const char * begin = select_str.data(); const char * end = begin + select_str.size(); ParserSelectQuery parser; - if (!parser.parse(begin, end, select, exprected)) + if (!parser.parse(begin, end, select, expected)) throw Poco::Exception("Cannot parse " + primary_expr_str); SharedPtr in = table->read(column_names, select, Settings(), stage)[0]; diff --git a/dbms/src/Storages/tests/pk_condition.cpp b/dbms/src/Storages/tests/pk_condition.cpp index 6b0751db975..e42f5bd482b 100644 --- a/dbms/src/Storages/tests/pk_condition.cpp +++ b/dbms/src/Storages/tests/pk_condition.cpp @@ -24,7 +24,7 @@ int main(int argc, const char ** argv) std::string query = "SELECT count() FROM pre.t WHERE (key > 9000 AND key < 100000 OR key > 200000 AND key < 1000000 OR key > 3000000 AND key < 8000000 OR key > 12000000)"; ASTPtr ast; IParser::Pos pos = &query[0]; - std::string error; + const char * error = ""; if (!parser.parse(pos, &query[0] + query.size(), ast, error)) { std::cout << "couldn't parse query" << std::endl; From d2df453682959e1290c2a22fda252a25c5805e21 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 17:26:44 +0400 Subject: [PATCH 07/23] dbms: improved performance a little [#METR-2944]. --- dbms/include/DB/DataTypes/DataTypeNested.h | 4 +--- dbms/src/DataTypes/DataTypeNested.cpp | 24 ++++------------------ 2 files changed, 5 insertions(+), 23 deletions(-) diff --git a/dbms/include/DB/DataTypes/DataTypeNested.h b/dbms/include/DB/DataTypes/DataTypeNested.h index c2b51695317..5184926b469 100644 --- a/dbms/include/DB/DataTypes/DataTypeNested.h +++ b/dbms/include/DB/DataTypes/DataTypeNested.h @@ -23,11 +23,9 @@ public: std::string getName() const; static std::string concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name); - static std::pair splitNestedName(const std::string & nested_name); - static bool isNestedName(const std::string & nested_name); /// Возвращает префикс имени до первой точки '.'. Или имя без изменений, если точки нет. static std::string extractNestedTableName(const std::string & nested_name); - /// Возвращает суффикс имени до первой точки справа '.'. Или имя без изменений, если точки нет. + /// Возвращает суффикс имени после первой точки справа '.'. Или имя без изменений, если точки нет. static std::string extractNestedColumnName(const std::string & nested_name); DataTypePtr clone() const diff --git a/dbms/src/DataTypes/DataTypeNested.cpp b/dbms/src/DataTypes/DataTypeNested.cpp index 1bdfdc6f090..c06c203f4ae 100644 --- a/dbms/src/DataTypes/DataTypeNested.cpp +++ b/dbms/src/DataTypes/DataTypeNested.cpp @@ -14,8 +14,6 @@ #include -#include - namespace DB { @@ -33,31 +31,17 @@ std::string DataTypeNested::concatenateNestedName(const std::string & nested_tab } -std::pair DataTypeNested::splitNestedName(const std::string & nested_name) -{ - Poco::StringTokenizer tokenizer(nested_name, "."); - return std::make_pair(tokenizer[0], tokenizer[1]); -} - - -bool DataTypeNested::isNestedName(const std::string & nested_name) -{ - Poco::StringTokenizer tokenizer(nested_name, "."); - return tokenizer.count() == 2 && tokenizer[0] != "" && tokenizer[1] != ""; -} - - std::string DataTypeNested::extractNestedTableName(const std::string & nested_name) { - Poco::StringTokenizer tokenizer(nested_name, "."); - return tokenizer[0]; + const char * pos = strchr(nested_name.data(), '.'); + return pos == nullptr ? nested_name : nested_name.substr(0, pos - nested_name.data()); } std::string DataTypeNested::extractNestedColumnName(const std::string & nested_name) { - Poco::StringTokenizer tokenizer(nested_name, "."); - return tokenizer[tokenizer.count() - 1]; + const char * pos = strrchr(nested_name.data(), '.'); + return pos == nullptr ? nested_name : nested_name.substr(pos - nested_name.data() + 1); } From eb2542009c60fb0c3afb1c5692816e8425bd475a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 18:47:04 +0400 Subject: [PATCH 08/23] dbms: improved performance of short queries [#METR-2944]. --- .../DB/Parsers/ExpressionElementParsers.h | 2 +- .../DB/Parsers/ExpressionListParsers.h | 126 ++++-------------- dbms/include/DB/Parsers/IParser.h | 4 +- dbms/src/Parsers/ExpressionListParsers.cpp | 102 ++++++++++---- dbms/src/Parsers/ParserCreateQuery.cpp | 2 +- dbms/src/Parsers/ParserInsertQuery.cpp | 2 +- 6 files changed, 111 insertions(+), 127 deletions(-) diff --git a/dbms/include/DB/Parsers/ExpressionElementParsers.h b/dbms/include/DB/Parsers/ExpressionElementParsers.h index 1e57e591db9..906ec79b61d 100644 --- a/dbms/include/DB/Parsers/ExpressionElementParsers.h +++ b/dbms/include/DB/Parsers/ExpressionElementParsers.h @@ -135,7 +135,7 @@ protected: class ParserWithOptionalAlias : public IParserBase { public: - ParserWithOptionalAlias(ParserPtr elem_parser_) : elem_parser(elem_parser_) {} + ParserWithOptionalAlias(ParserPtr && elem_parser_) : elem_parser(std::move(elem_parser_)) {} protected: ParserPtr elem_parser; diff --git a/dbms/include/DB/Parsers/ExpressionListParsers.h b/dbms/include/DB/Parsers/ExpressionListParsers.h index b99f93b675c..011d1ab330c 100644 --- a/dbms/include/DB/Parsers/ExpressionListParsers.h +++ b/dbms/include/DB/Parsers/ExpressionListParsers.h @@ -2,8 +2,6 @@ #include -#include - #include #include @@ -11,18 +9,18 @@ namespace DB { -/** Оператор и соответствующая ему функция. Например, "+" -> "plus" - * Не std::map, так как порядок парсинга операторов задаётся явно и может отличаться от алфавитного. +/** Идущие подряд пары строк: оператор и соответствующая ему функция. Например, "+" -> "plus". + * Порядок парсинга операторов имеет значение. */ -typedef std::list > Operators_t; +typedef const char ** Operators_t; /** Список элементов, разделённых чем-либо. */ class ParserList : public IParserBase { public: - ParserList(ParserPtr elem_parser_, ParserPtr separator_parser_, bool allow_empty_ = true) - : elem_parser(elem_parser_), separator_parser(separator_parser_), allow_empty(allow_empty_) + ParserList(ParserPtr && elem_parser_, ParserPtr && separator_parser_, bool allow_empty_ = true) + : elem_parser(std::move(elem_parser_)), separator_parser(std::move(separator_parser_)), allow_empty(allow_empty_) { } protected: @@ -47,8 +45,8 @@ private: public: /** operators_ - допустимые операторы и соответствующие им функции */ - ParserLeftAssociativeBinaryOperatorList(const Operators_t & operators_, ParserPtr elem_parser_) - : operators(operators_), elem_parser(elem_parser_) + ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && elem_parser_) + : operators(operators_), elem_parser(std::move(elem_parser_)) { } @@ -66,12 +64,12 @@ class ParserVariableArityOperatorList : public IParserBase { private: ParserString infix_parser; - String function_name; + const char * function_name; ParserPtr elem_parser; public: - ParserVariableArityOperatorList(const char * infix_, const String & function_, ParserPtr elem_parser_) - : infix_parser(infix_, true, true), function_name(function_), elem_parser(elem_parser_) + ParserVariableArityOperatorList(const char * infix_, const char * function_, ParserPtr && elem_parser_) + : infix_parser(infix_, true, true), function_name(function_), elem_parser(std::move(elem_parser_)) { } @@ -94,8 +92,8 @@ private: public: /** operators_ - допустимые операторы и соответствующие им функции */ - ParserPrefixUnaryOperatorExpression(const Operators_t & operators_, ParserPtr elem_parser_) - : operators(operators_), elem_parser(elem_parser_) + ParserPrefixUnaryOperatorExpression(Operators_t operators_, ParserPtr && elem_parser_) + : operators(operators_), elem_parser(std::move(elem_parser_)) { } @@ -108,7 +106,7 @@ protected: class ParserAccessExpression : public IParserBase { private: - ParserPtr elem_parser; + static const char * operators[]; ParserLeftAssociativeBinaryOperatorList operator_parser; public: ParserAccessExpression(); @@ -126,14 +124,8 @@ protected: class ParserUnaryMinusExpression : public IParserBase { private: - ParserPtr elem_parser; - ParserPrefixUnaryOperatorExpression operator_parser; -public: - ParserUnaryMinusExpression() - : elem_parser(new ParserAccessExpression), - operator_parser(boost::assign::map_list_of("-", "negate"), elem_parser) - { - } + static const char * operators[]; + ParserPrefixUnaryOperatorExpression operator_parser {operators, ParserPtr(new ParserAccessExpression)}; protected: const char * getName() const { return "unary minus expression"; } @@ -145,19 +137,9 @@ protected: class ParserMultiplicativeExpression : public IParserBase { private: - ParserPtr elem_parser; - ParserLeftAssociativeBinaryOperatorList operator_parser; -public: - ParserMultiplicativeExpression() - : elem_parser(new ParserUnaryMinusExpression), - operator_parser(boost::assign::map_list_of - ("*", "multiply") - ("/", "divide") - ("%", "modulo"), - elem_parser) - { - } - + static const char * operators[]; + ParserLeftAssociativeBinaryOperatorList operator_parser {operators, ParserPtr(new ParserUnaryMinusExpression)}; + protected: const char * getName() const { return "multiplicative expression"; } @@ -171,17 +153,8 @@ protected: class ParserAdditiveExpression : public IParserBase { private: - ParserPtr elem_parser; - ParserLeftAssociativeBinaryOperatorList operator_parser; -public: - ParserAdditiveExpression() - : elem_parser(new ParserMultiplicativeExpression), - operator_parser(boost::assign::map_list_of - ("+", "plus") - ("-", "minus"), - elem_parser) - { - } + static const char * operators[]; + ParserLeftAssociativeBinaryOperatorList operator_parser {operators, ParserPtr(new ParserMultiplicativeExpression)}; protected: const char * getName() const { return "additive expression"; } @@ -196,27 +169,8 @@ protected: class ParserComparisonExpression : public IParserBase { private: - ParserPtr elem_parser; - ParserLeftAssociativeBinaryOperatorList operator_parser; -public: - ParserComparisonExpression() - : elem_parser(new ParserAdditiveExpression), - operator_parser(boost::assign::map_list_of - ("==", "equals") - ("!=", "notEquals") - ("<>", "notEquals") - ("<=", "lessOrEquals") - (">=", "greaterOrEquals") - ("<", "less") - (">", "greater") - ("=", "equals") - ("LIKE", "like") - ("NOT LIKE", "notLike") - ("IN", "in") - ("NOT IN", "notIn"), - elem_parser) - { - } + static const char * operators[]; + ParserLeftAssociativeBinaryOperatorList operator_parser {operators, ParserPtr(new ParserAdditiveExpression)}; protected: const char * getName() const { return "comparison expression"; } @@ -231,15 +185,9 @@ protected: class ParserLogicalNotExpression : public IParserBase { private: - ParserPtr elem_parser; - ParserPrefixUnaryOperatorExpression operator_parser; -public: - ParserLogicalNotExpression() - : elem_parser(new ParserComparisonExpression), - operator_parser(boost::assign::map_list_of("NOT", "not"), elem_parser) - { - } - + static const char * operators[]; + ParserPrefixUnaryOperatorExpression operator_parser {operators, ParserPtr(new ParserComparisonExpression)}; + protected: const char * getName() const { return "logical-NOT expression"; } @@ -253,12 +201,10 @@ protected: class ParserLogicalAndExpression : public IParserBase { private: - ParserPtr elem_parser; ParserVariableArityOperatorList operator_parser; public: ParserLogicalAndExpression() - : elem_parser(new ParserLogicalNotExpression), - operator_parser("AND", "and", elem_parser) + : operator_parser("AND", "and", ParserPtr(new ParserLogicalNotExpression)) { } @@ -275,12 +221,10 @@ protected: class ParserLogicalOrExpression : public IParserBase { private: - ParserPtr elem_parser; ParserVariableArityOperatorList operator_parser; public: ParserLogicalOrExpression() - : elem_parser(new ParserLogicalAndExpression), - operator_parser("OR", "or", elem_parser) + : operator_parser("OR", "or", ParserPtr(new ParserLogicalAndExpression)) { } @@ -300,13 +244,7 @@ protected: class ParserTernaryOperatorExpression : public IParserBase { private: - ParserPtr elem_parser; - -public: - ParserTernaryOperatorExpression() - : elem_parser(new ParserLogicalOrExpression) - { - } + ParserLogicalOrExpression elem_parser; protected: const char * getName() const { return "expression with ternary operator"; } @@ -318,13 +256,7 @@ protected: class ParserLambdaExpression : public IParserBase { private: - ParserPtr elem_parser; - -public: - ParserLambdaExpression() - : elem_parser(new ParserTernaryOperatorExpression) - { - } + ParserTernaryOperatorExpression elem_parser; protected: const char * getName() const { return "lambda expression"; } diff --git a/dbms/include/DB/Parsers/IParser.h b/dbms/include/DB/Parsers/IParser.h index 52d9bc88ec7..751dbe973c9 100644 --- a/dbms/include/DB/Parsers/IParser.h +++ b/dbms/include/DB/Parsers/IParser.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -63,6 +63,6 @@ public: virtual ~IParser() {} }; -typedef SharedPtr ParserPtr; +typedef std::unique_ptr ParserPtr; } diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index f03b9c18759..a49e56f17a8 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -12,6 +12,59 @@ namespace DB { +const char * ParserMultiplicativeExpression::operators[] = +{ + "*", "multiply", + "/", "divide", + "%", "modulo", + nullptr, nullptr +}; + +const char * ParserUnaryMinusExpression::operators[] = +{ + "-", "negate", + nullptr, nullptr +}; + +const char * ParserAdditiveExpression::operators[] = +{ + "+", "plus", + "-", "minus", + nullptr, nullptr +}; + +const char * ParserComparisonExpression::operators[] = +{ + "==", "equals", + "!=", "notEquals", + "<>", "notEquals", + "<=", "lessOrEquals", + ">=", "greaterOrEquals", + "<", "less", + ">", "greater", + "=", "equals", + "LIKE", "like", + "NOT LIKE", "notLike", + "IN", "in", + "NOT IN", "notIn", + nullptr, nullptr +}; + +const char * ParserLogicalNotExpression::operators[] = +{ + "NOT", "not", + nullptr, nullptr +}; + +const char * ParserAccessExpression::operators[] = +{ + ".", "tupleElement", + "[", "arrayElement", + nullptr, nullptr +}; + + + bool ParserList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { bool first = true; @@ -75,15 +128,16 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP /// пробуем найти какой-нибудь из допустимых операторов Pos begin = pos; - Operators_t::const_iterator it; - for (it = operators.begin(); it != operators.end(); ++it) + + const char ** it; + for (it = operators; *it; it += 2) { - ParserString op(it->first, true, true); + ParserString op(it[0], true, true); if (op.ignore(pos, end, expected)) break; } - if (it == operators.end()) + if (!*it) break; ws.ignore(pos, end); @@ -105,7 +159,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP /// первым аргументом функции будет предыдущий элемент, вторым - следующий function.range.first = begin; function.range.second = pos; - function.name = it->second; + function.name = it[1]; function.arguments = exp_list_node; function.children.push_back(exp_list_node); @@ -117,7 +171,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP /** специальное исключение для оператора доступа к элементу массива x[y], который * содержит инфиксную часть '[' и суффиксную ']' (задаётся в виде '[') */ - if (0 == strcmp(it->first, "[")) + if (0 == strcmp(it[0], "[")) { ParserString rest_p("]"); @@ -191,7 +245,7 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & nod Pos begin = pos; - if (!elem_parser->parse(pos, end, elem_cond, expected)) + if (!elem_parser.parse(pos, end, elem_cond, expected)) return false; ws.ignore(pos, end); @@ -202,7 +256,7 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & nod { ws.ignore(pos, end); - if (!elem_parser->parse(pos, end, elem_then, expected)) + if (!elem_parser.parse(pos, end, elem_then, expected)) return false; ws.ignore(pos, end); @@ -212,7 +266,7 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & nod ws.ignore(pos, end); - if (!elem_parser->parse(pos, end, elem_else, expected)) + if (!elem_parser.parse(pos, end, elem_else, expected)) return false; /// функция, соответствующая оператору @@ -266,7 +320,7 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, const was_open = true; } - if (!ParserList(new ParserIdentifier, new ParserString(",")).parse(pos, end, inner_arguments, expected)) + if (!ParserList(ParserPtr(new ParserIdentifier), ParserPtr(new ParserString(","))).parse(pos, end, inner_arguments, expected)) break; ws.ignore(pos, end, expected); @@ -281,7 +335,7 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, const break; ws.ignore(pos, end, expected); - if (!elem_parser->parse(pos, end, expression, expected)) + if (!elem_parser.parse(pos, end, expression, expected)) { pos = begin; return false; @@ -310,7 +364,7 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, const while (false); pos = begin; - return elem_parser->parse(pos, end, node, expected); + return elem_parser.parse(pos, end, node, expected); } @@ -320,10 +374,10 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & /// пробуем найти какой-нибудь из допустимых операторов Pos begin = pos; - Operators_t::const_iterator it; - for (it = operators.begin(); it != operators.end(); ++it) + const char ** it; + for (it = operators; *it; it += 2) { - ParserString op(it->first, true, true); + ParserString op(it[0], true, true); if (op.ignore(pos, end, expected)) break; } @@ -334,7 +388,7 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & if (!elem_parser->parse(pos, end, elem, expected)) return false; - if (it == operators.end()) + if (!*it) node = elem; else { @@ -350,7 +404,7 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & function.range.first = begin; function.range.second = pos; - function.name = it->second; + function.name = it[1]; function.arguments = exp_list_node; function.children.push_back(exp_list_node); @@ -385,24 +439,22 @@ bool ParserUnaryMinusExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, co ParserAccessExpression::ParserAccessExpression() - : elem_parser(new ParserExpressionElement), - operator_parser(boost::assign::map_list_of - (".", "tupleElement") - ("[", "arrayElement"), - elem_parser) + : operator_parser( + operators, + ParserPtr(new ParserExpressionElement)) { } ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias() - : impl(new ParserWithOptionalAlias(new ParserLambdaExpression)) + : impl(new ParserWithOptionalAlias(ParserPtr(new ParserLambdaExpression))) { } bool ParserExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { - return ParserList(new ParserExpressionWithOptionalAlias, new ParserString(",")).parse(pos, end, node, expected); + return ParserList(ParserPtr(new ParserExpressionWithOptionalAlias), ParserPtr(new ParserString(","))).parse(pos, end, node, expected); } @@ -415,7 +467,7 @@ bool ParserNotEmptyExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, bool ParserOrderByExpressionList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { - return ParserList(new ParserOrderByElement, new ParserString(","), false).parse(pos, end, node, expected); + return ParserList(ParserPtr(new ParserOrderByElement), ParserPtr(new ParserString(",")), false).parse(pos, end, node, expected); } diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 5ecbff94a0c..23b95fd6f28 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -130,7 +130,7 @@ bool ParserNameTypePair::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char bool ParserNameTypePairList::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char *& expected) { - return ParserList(new ParserNameTypePair, new ParserString(","), false).parse(pos, end, node, expected); + return ParserList(ParserPtr(new ParserNameTypePair), ParserPtr(new ParserString(",")), false).parse(pos, end, node, expected); } diff --git a/dbms/src/Parsers/ParserInsertQuery.cpp b/dbms/src/Parsers/ParserInsertQuery.cpp index 88cc72f30a6..e33d3902345 100644 --- a/dbms/src/Parsers/ParserInsertQuery.cpp +++ b/dbms/src/Parsers/ParserInsertQuery.cpp @@ -27,7 +27,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char ParserString s_lparen("("); ParserString s_rparen(")"); ParserIdentifier name_p; - ParserList columns_p(new ParserIdentifier, new ParserString(","), false); + ParserList columns_p(ParserPtr(new ParserIdentifier), ParserPtr(new ParserString(",")), false); ASTPtr database; ASTPtr table; From 07b16db5b2f90a448b29dee6c752abaa43122116 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 18:49:36 +0400 Subject: [PATCH 09/23] dbms: addition to prev. revision [#METR-2944]. --- dbms/src/Interpreters/tests/expression_analyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index f4e67ca95cc..abe55518ba6 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -29,7 +29,7 @@ int main(int argc, char ** argv) } ASTPtr root; - ParserPtr parsers[] = {new ParserSelectQuery, new ParserExpressionList}; + ParserPtr parsers[] = {ParserPtr(new ParserSelectQuery), ParserPtr(new ParserExpressionList)}; for (size_t i = 0; i < sizeof(parsers)/sizeof(parsers[0]); ++i) { IParser & parser = *parsers[i]; From c23feaa671cadc1b664130cbb665392dd13d9419 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 19:58:30 +0400 Subject: [PATCH 10/23] dbms: tiny modification [#METR-2944]. --- dbms/src/IO/WriteBufferFromPocoSocket.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromPocoSocket.cpp b/dbms/src/IO/WriteBufferFromPocoSocket.cpp index 874e9760693..977359d7a8c 100644 --- a/dbms/src/IO/WriteBufferFromPocoSocket.cpp +++ b/dbms/src/IO/WriteBufferFromPocoSocket.cpp @@ -26,7 +26,7 @@ void WriteBufferFromPocoSocket::nextImpl() } catch (const Poco::Net::NetException & e) { - throw Exception(e.displayText() + "while writing to socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); + throw Exception(e.displayText() + " while writing to socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); } catch (const Poco::TimeoutException & e) { @@ -34,7 +34,7 @@ void WriteBufferFromPocoSocket::nextImpl() } catch (const Poco::IOException & e) { - throw Exception(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); + throw Exception(e.displayText(), " while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); } if (res < 0) From a022302ae4d6c25dab3c0066b7031ff2fb952ddc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 22:01:26 +0400 Subject: [PATCH 11/23] dbms: better pretty-printing [#METR-2944]. --- dbms/include/DB/Parsers/formatAST.h | 52 +++-- dbms/src/Parsers/ExpressionListParsers.cpp | 12 +- dbms/src/Parsers/formatAST.cpp | 250 ++++++++++++++++----- 3 files changed, 232 insertions(+), 82 deletions(-) diff --git a/dbms/include/DB/Parsers/formatAST.h b/dbms/include/DB/Parsers/formatAST.h index 2600210f129..3217125902b 100644 --- a/dbms/include/DB/Parsers/formatAST.h +++ b/dbms/include/DB/Parsers/formatAST.h @@ -31,31 +31,35 @@ namespace DB /** Берёт синтаксическое дерево и превращает его обратно в текст. * В случае запроса INSERT, данные будут отсутствовать. */ -void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); +void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); -void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); -void formatAST(const ASTShowProcesslistQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false); +void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); +void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); + +void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s, + size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); + +void formatAST(const ASTShowProcesslistQuery & ast, std::ostream & s, + size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false); String formatColumnsForCreateQuery(NamesAndTypesList & columns); diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index a49e56f17a8..ee71f89a92b 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -17,20 +17,20 @@ const char * ParserMultiplicativeExpression::operators[] = "*", "multiply", "/", "divide", "%", "modulo", - nullptr, nullptr + nullptr }; const char * ParserUnaryMinusExpression::operators[] = { "-", "negate", - nullptr, nullptr + nullptr }; const char * ParserAdditiveExpression::operators[] = { "+", "plus", "-", "minus", - nullptr, nullptr + nullptr }; const char * ParserComparisonExpression::operators[] = @@ -47,20 +47,20 @@ const char * ParserComparisonExpression::operators[] = "NOT LIKE", "notLike", "IN", "in", "NOT IN", "notIn", - nullptr, nullptr + nullptr }; const char * ParserLogicalNotExpression::operators[] = { "NOT", "not", - nullptr, nullptr + nullptr }; const char * ParserAccessExpression::operators[] = { ".", "tupleElement", "[", "arrayElement", - nullptr, nullptr + nullptr }; diff --git a/dbms/src/Parsers/formatAST.cpp b/dbms/src/Parsers/formatAST.cpp index 4707a78c8f9..cd10dd161ea 100644 --- a/dbms/src/Parsers/formatAST.cpp +++ b/dbms/src/Parsers/formatAST.cpp @@ -22,6 +22,7 @@ namespace DB static const char * hilite_keyword = "\033[1;37m"; static const char * hilite_identifier = "\033[0;36m"; static const char * hilite_function = "\033[0;33m"; +static const char * hilite_operator = "\033[1;33m"; static const char * hilite_alias = "\033[0;32m"; static const char * hilite_none = "\033[0m"; @@ -38,12 +39,12 @@ String backQuoteIfNeed(const String & x) } -void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { #define DISPATCH(NAME) \ else if (const AST ## NAME * concrete = dynamic_cast(&ast)) \ - formatAST(*concrete, s, indent, hilite, one_line); + formatAST(*concrete, s, indent, hilite, one_line, need_parens); if (false) {} DISPATCH(SelectQuery) @@ -76,7 +77,37 @@ void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, b } -void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) +{ + for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it) + { + if (it != ast.children.begin()) + s << ", "; + + formatAST(**it, s, indent, hilite, one_line, need_parens); + } +} + +/** Вывести список выражений в секциях запроса SELECT - по одному выражению на строку. + */ +static void formatExpressionListMultiline(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite) +{ + std::string indent_str = "\n" + std::string(4 * (indent + 1), ' '); + + for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it) + { + if (it != ast.children.begin()) + s << ", "; + + if (ast.children.size() > 1) + s << indent_str; + + formatAST(**it, s, indent + 1, hilite, false); + } +} + + +void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { std::string nl_or_nothing = one_line ? "" : "\n"; @@ -84,7 +115,9 @@ void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bo std::string nl_or_ws = one_line ? " " : "\n"; s << (hilite ? hilite_keyword : "") << indent_str << "SELECT " << (ast.distinct ? "DISTINCT " : "") << (hilite ? hilite_none : ""); - formatAST(*ast.select_expression_list, s, indent, hilite, one_line); + one_line + ? formatAST(*ast.select_expression_list, s, indent, hilite, one_line) + : formatExpressionListMultiline(dynamic_cast(*ast.select_expression_list), s, indent, hilite); if (ast.table) { @@ -116,7 +149,9 @@ void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bo if (ast.array_join_expression_list) { s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "ARRAY JOIN " << (hilite ? hilite_none : ""); - formatAST(*ast.array_join_expression_list, s, indent, hilite, one_line); + one_line + ? formatAST(*ast.array_join_expression_list, s, indent, hilite, one_line) + : formatExpressionListMultiline(dynamic_cast(*ast.array_join_expression_list), s, indent, hilite); } if (ast.final) @@ -133,7 +168,9 @@ void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bo if (ast.prewhere_expression) { s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "PREWHERE " << (hilite ? hilite_none : ""); - formatAST(*ast.prewhere_expression, s, indent, hilite, one_line); + one_line + ? formatAST(*ast.prewhere_expression, s, indent, hilite, one_line) + : formatExpressionListMultiline(dynamic_cast(*ast.prewhere_expression), s, indent, hilite); } if (ast.where_expression) @@ -145,10 +182,12 @@ void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bo if (ast.group_expression_list) { s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "GROUP BY " << (hilite ? hilite_none : ""); - formatAST(*ast.group_expression_list, s, indent, hilite, one_line); + one_line + ? formatAST(*ast.group_expression_list, s, indent, hilite, one_line) + : formatExpressionListMultiline(dynamic_cast(*ast.group_expression_list), s, indent, hilite); if (ast.group_by_with_totals) - s << (hilite ? hilite_keyword : "") << " WITH TOTALS" << (hilite ? hilite_none : ""); + s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << (one_line ? "" : " ") << "WITH TOTALS" << (hilite ? hilite_none : ""); } if (ast.having_expression) @@ -160,7 +199,9 @@ void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bo if (ast.order_expression_list) { s << (hilite ? hilite_keyword : "") << nl_or_ws << indent_str << "ORDER BY " << (hilite ? hilite_none : ""); - formatAST(*ast.order_expression_list, s, indent, hilite, one_line); + one_line + ? formatAST(*ast.order_expression_list, s, indent, hilite, one_line) + : formatExpressionListMultiline(dynamic_cast(*ast.order_expression_list), s, indent, hilite); } if (ast.limit_length) @@ -181,7 +222,7 @@ void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bo } } -void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { std::string nl_or_nothing = one_line ? "" : "\n"; @@ -190,7 +231,7 @@ void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent, bool s << nl_or_nothing << ")"; } -void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { std::string nl_or_ws = one_line ? " " : "\n"; @@ -249,7 +290,7 @@ void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent, bo } } -void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { if (ast.table.empty() && !ast.database.empty()) { @@ -261,13 +302,13 @@ void formatAST(const ASTDropQuery & ast, std::ostream & s, size_t indent, boo << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); } -void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTOptimizeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << (hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (hilite ? hilite_none : "") << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); } -void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << (hilite ? hilite_keyword : "") << name << " " << (hilite ? hilite_none : "") << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); @@ -281,22 +322,22 @@ void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::os } } -void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTExistsQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { formatAST(static_cast(ast), "EXISTS TABLE", s, indent, hilite, one_line); } -void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTDescribeQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { formatAST(static_cast(ast), "DESCRIBE TABLE", s, indent, hilite, one_line); } -void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTShowCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { formatAST(static_cast(ast), "SHOW CREATE TABLE", s, indent, hilite, one_line); } -void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << (hilite ? hilite_keyword : "") << "RENAME TABLE " << (hilite ? hilite_none : ""); @@ -311,7 +352,7 @@ void formatAST(const ASTRenameQuery & ast, std::ostream & s, size_t indent, bo } } -void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << (hilite ? hilite_keyword : "") << "SET " << (ast.global ? "GLOBAL " : "") << (hilite ? hilite_none : ""); @@ -324,7 +365,7 @@ void formatAST(const ASTSetQuery & ast, std::ostream & s, size_t indent, bool } } -void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { if (ast.databases) { @@ -352,19 +393,19 @@ void formatAST(const ASTShowTablesQuery & ast, std::ostream & s, size_t indent, } } -void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTUseQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << (hilite ? hilite_keyword : "") << "USE " << (hilite ? hilite_none : "") << backQuoteIfNeed(ast.database); return; } -void formatAST(const ASTShowProcesslistQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTShowProcesslistQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << (hilite ? hilite_keyword : "") << "SHOW PROCESSLIST" << (hilite ? hilite_none : ""); return; } -void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << (hilite ? hilite_keyword : "") << "INSERT INTO " << (hilite ? hilite_none : "") << (!ast.database.empty() ? backQuoteIfNeed(ast.database) + "." : "") << backQuoteIfNeed(ast.table); @@ -394,16 +435,6 @@ void formatAST(const ASTInsertQuery & ast, std::ostream & s, size_t indent, bo } } -void formatAST(const ASTExpressionList & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) -{ - for (ASTs::const_iterator it = ast.children.begin(); it != ast.children.end(); ++it) - { - if (it != ast.children.begin()) - s << ", "; - formatAST(**it, s, indent, hilite, one_line); - } -} - static void writeAlias(const String & name, std::ostream & s, bool hilite, bool one_line) { s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_alias : ""); @@ -415,31 +446,146 @@ static void writeAlias(const String & name, std::ostream & s, bool hilite, bool s << (hilite ? hilite_none : ""); } -void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { - s << (hilite ? hilite_function : "") << ast.name; - - if (ast.parameters) + /// Стоит ли записать эту функцию в виде оператора? + bool written = false; + if (ast.arguments && !ast.parameters) { - s << '(' << (hilite ? hilite_none : ""); - formatAST(*ast.parameters, s, indent, hilite, one_line); - s << (hilite ? hilite_function : "") << ')'; + if (ast.arguments->children.size() == 1) + { + const char * operators[] = + { + "negate", "-", + "not", "NOT ", + nullptr + }; + + for (const char ** func = operators; *func; func += 2) + { + if (0 == strcmp(ast.name.c_str(), func[0])) + { + s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : ""); + formatAST(*ast.arguments, s, indent, hilite, one_line, true); + written = true; + } + } + } + + /** need_parens - нужны ли скобки вокруг выражения с оператором. + * Они нужны, только если это выражение входит в другое выражение с оператором. + */ + + if (!written && ast.arguments->children.size() == 2) + { + const char * operators[] = + { + "multiply", " * ", + "divide", " / ", + "modulo", " % ", + "plus", " + ", + "minus", " - ", + "notEquals", " != ", + "lessOrEquals", " <= ", + "greaterOrEquals", " >= ", + "less", " < ", + "greater", " > ", + "equals", " = ", + "like", " LIKE ", + "notLike", " NOT LIKE ", + "in", " IN ", + "notIn", " NOT IN ", + nullptr + }; + + for (const char ** func = operators; *func; func += 2) + { + if (0 == strcmp(ast.name.c_str(), func[0])) + { + if (need_parens) + s << '('; + formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true); + s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : ""); + formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true); + if (need_parens) + s << ')'; + written = true; + } + } + + if (!written && 0 == strcmp(ast.name.c_str(), "arrayElement")) + { + formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true); + s << (hilite ? hilite_operator : "") << '[' << (hilite ? hilite_none : ""); + formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true); + s << (hilite ? hilite_operator : "") << ']' << (hilite ? hilite_none : ""); + written = true; + } + + if (!written && 0 == strcmp(ast.name.c_str(), "tupleElement")) + { + formatAST(*ast.arguments->children[0], s, indent, hilite, one_line, true); + s << (hilite ? hilite_operator : "") << "." << (hilite ? hilite_none : ""); + formatAST(*ast.arguments->children[1], s, indent, hilite, one_line, true); + written = true; + } + } + + if (!written && ast.arguments->children.size() >= 2) + { + const char * operators[] = + { + "and", " AND ", + "or", " OR ", + nullptr + }; + + for (const char ** func = operators; *func; func += 2) + { + if (0 == strcmp(ast.name.c_str(), func[0])) + { + if (need_parens) + s << '('; + for (size_t i = 0; i < ast.arguments->children.size(); ++i) + { + if (i != 0) + s << (hilite ? hilite_operator : "") << func[1] << (hilite ? hilite_none : ""); + formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, true); + } + if (need_parens) + s << ')'; + written = true; + } + } + } } - if (ast.arguments) + if (!written) { - s << '(' << (hilite ? hilite_none : ""); - formatAST(*ast.arguments, s, indent, hilite, one_line); - s << (hilite ? hilite_function : "") << ')'; + s << (hilite ? hilite_function : "") << ast.name; + + if (ast.parameters) + { + s << '(' << (hilite ? hilite_none : ""); + formatAST(*ast.parameters, s, indent, hilite, one_line); + s << (hilite ? hilite_function : "") << ')'; + } + + if (ast.arguments) + { + s << '(' << (hilite ? hilite_none : ""); + formatAST(*ast.arguments, s, indent, hilite, one_line); + s << (hilite ? hilite_function : "") << ')'; + } + + s << (hilite ? hilite_none : ""); } - - s << (hilite ? hilite_none : ""); if (!ast.alias.empty()) writeAlias(ast.alias, s, hilite, one_line); } -void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << (hilite ? hilite_identifier : ""); @@ -453,7 +599,7 @@ void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bo writeAlias(ast.alias, s, hilite, one_line); } -void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << apply_visitor(FieldVisitorToString(), ast.value); @@ -461,7 +607,7 @@ void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent, bool writeAlias(ast.alias, s, hilite, one_line); } -void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); std::string nl_or_ws = one_line ? " " : "\n"; @@ -470,12 +616,12 @@ void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent, bo formatAST(*ast.type, s, indent, hilite, one_line); } -void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTAsterisk & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { s << "*"; } -void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { formatAST(*ast.children.front(), s, indent, hilite, one_line); s << (hilite ? hilite_keyword : "") << (ast.direction == -1 ? " DESC" : " ASC") << (hilite ? hilite_none : ""); @@ -486,7 +632,7 @@ void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent, } } -void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line) +void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { std::string nl_or_nothing = one_line ? "" : "\n"; From 780e75252e4b4e4ca46c37e1a30dfe75680834f7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Mar 2014 22:10:44 +0400 Subject: [PATCH 12/23] Addition to prev. revision [#METR-2944]. --- dbms/src/Parsers/formatAST.cpp | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/dbms/src/Parsers/formatAST.cpp b/dbms/src/Parsers/formatAST.cpp index cd10dd161ea..3fcfe94ae1e 100644 --- a/dbms/src/Parsers/formatAST.cpp +++ b/dbms/src/Parsers/formatAST.cpp @@ -557,6 +557,32 @@ void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent, bool written = true; } } + + if (!written && 0 == strcmp(ast.name.c_str(), "array")) + { + s << '['; + for (size_t i = 0; i < ast.arguments->children.size(); ++i) + { + if (i != 0) + s << ", "; + formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, false); + } + s << ']'; + written = true; + } + + if (!written && 0 == strcmp(ast.name.c_str(), "tuple")) + { + s << '('; + for (size_t i = 0; i < ast.arguments->children.size(); ++i) + { + if (i != 0) + s << ", "; + formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, false); + } + s << ')'; + written = true; + } } } From 00b10d30df41b5b1f79ad65e607c30ba9e712293 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Mar 2014 11:32:47 +0400 Subject: [PATCH 13/23] removed obsolete comment. [#METR-2807] --- dbms/include/DB/Common/LRUCache.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/include/DB/Common/LRUCache.h b/dbms/include/DB/Common/LRUCache.h index f8575891085..33dc628babe 100644 --- a/dbms/include/DB/Common/LRUCache.h +++ b/dbms/include/DB/Common/LRUCache.h @@ -88,7 +88,6 @@ public: void getStats(size_t & out_hits, size_t & out_misses) const { Poco::ScopedLock lock(mutex); - /// Синхронизация не нужна. out_hits = hits; out_misses = misses; } From 5df33181345bad3ce618ad2944dd86a9f244a1bd Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Wed, 12 Mar 2014 14:28:25 +0400 Subject: [PATCH 14/23] dbms: fixed possible race condition in alter [#METR-10242] --- .../Storages/MergeTree/MergeTreeBlockInputStream.h | 7 +++++-- .../Storages/MergeTree/MergeTreeBlockOutputStream.h | 7 ++++--- dbms/src/Storages/StorageMergeTree.cpp | 12 ++++++------ 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeBlockInputStream.h index 09931dba6e7..a298e6f2482 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeBlockInputStream.h @@ -20,13 +20,14 @@ public: size_t block_size_, const Names & column_names_, StorageMergeTree & storage_, const StorageMergeTree::DataPartPtr & owned_data_part_, const MarkRanges & mark_ranges_, StoragePtr owned_storage, bool use_uncompressed_cache_, - ExpressionActionsPtr prewhere_actions_, String prewhere_column_) + ExpressionActionsPtr prewhere_actions_, String prewhere_column_, bool take_read_lock) : IProfilingBlockInputStream(owned_storage), path(path_), block_size(block_size_), column_names(column_names_), storage(storage_), owned_data_part(owned_data_part_), all_mark_ranges(mark_ranges_), remaining_mark_ranges(mark_ranges_), use_uncompressed_cache(use_uncompressed_cache_), - prewhere_actions(prewhere_actions_), prewhere_column(prewhere_column_) + prewhere_actions(prewhere_actions_), prewhere_column(prewhere_column_), + lock(take_read_lock ? new Poco::ScopedReadRWLock(storage.read_lock) : NULL) { std::reverse(remaining_mark_ranges.begin(), remaining_mark_ranges.end()); @@ -332,6 +333,8 @@ private: ExpressionActionsPtr prewhere_actions; String prewhere_column; bool remove_prewhere_column; + + std::unique_ptr lock; }; } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeBlockOutputStream.h index f74e82860c0..ab1e24f502d 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeBlockOutputStream.h @@ -16,14 +16,13 @@ namespace DB class MergeTreeBlockOutputStream : public IBlockOutputStream { public: - MergeTreeBlockOutputStream(StoragePtr owned_storage) : IBlockOutputStream(owned_storage), storage(dynamic_cast(*owned_storage)), flags(O_TRUNC | O_CREAT | O_WRONLY) + MergeTreeBlockOutputStream(StoragePtr owned_storage, bool take_lock) : IBlockOutputStream(owned_storage), storage(dynamic_cast(*owned_storage)), flags(O_TRUNC | O_CREAT | O_WRONLY), + lock(take_lock ? new Poco::ScopedReadRWLock(storage.write_lock) : NULL) { } void write(const Block & block) { - Poco::ScopedReadRWLock write_lock(storage.write_lock); - storage.check(block, true); DateLUTSingleton & date_lut = DateLUTSingleton::instance(); @@ -86,6 +85,8 @@ private: const int flags; + std::unique_ptr lock; + struct BlockWithDateInterval { Block block; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 537a9cf0f89..c90d111c3fc 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -146,7 +146,7 @@ StorageMergeTree::~StorageMergeTree() BlockOutputStreamPtr StorageMergeTree::write(ASTPtr query) { - return new MergeTreeBlockOutputStream(thisPtr()); + return new MergeTreeBlockOutputStream(thisPtr(), true); } @@ -416,7 +416,7 @@ BlockInputStreams StorageMergeTree::spreadMarkRangesAmongThreads( streams.push_back(new MergeTreeBlockInputStream( full_path + part.data_part->name + '/', max_block_size, column_names, *this, part.data_part, part.ranges, thisPtr(), use_uncompressed_cache, - prewhere_actions, prewhere_column)); + prewhere_actions, prewhere_column, true)); need_marks -= marks_in_part; parts.pop_back(); sum_marks_in_parts.pop_back(); @@ -446,7 +446,7 @@ BlockInputStreams StorageMergeTree::spreadMarkRangesAmongThreads( streams.push_back(new MergeTreeBlockInputStream( full_path + part.data_part->name + '/', max_block_size, column_names, *this, part.data_part, ranges_to_get_from_part, thisPtr(), use_uncompressed_cache, - prewhere_actions, prewhere_column)); + prewhere_actions, prewhere_column, true)); } if (streams.size() == 1) @@ -494,7 +494,7 @@ BlockInputStreams StorageMergeTree::spreadMarkRangesAmongThreadsFinal( BlockInputStreamPtr source_stream = new MergeTreeBlockInputStream( full_path + part.data_part->name + '/', max_block_size, column_names, *this, part.data_part, part.ranges, thisPtr(), use_uncompressed_cache, - prewhere_actions, prewhere_column); + prewhere_actions, prewhere_column, true); to_collapse.push_back(new ExpressionBlockInputStream(source_stream, primary_expr)); } @@ -1098,7 +1098,7 @@ void StorageMergeTree::mergeParts(Poco::SharedPtr & MarkRanges ranges(1, MarkRange(0, parts[i]->size)); src_streams.push_back(new ExpressionBlockInputStream(new MergeTreeBlockInputStream( full_path + parts[i]->name + '/', DEFAULT_MERGE_BLOCK_SIZE, all_column_names, *this, parts[i], ranges, - StoragePtr(), false, NULL, ""), primary_expr)); + StoragePtr(), false, NULL, "", true), primary_expr)); } /// Порядок потоков важен: при совпадении ключа элементы идут в порядке номера потока-источника. @@ -1301,7 +1301,7 @@ void StorageMergeTree::alter(const ASTAlterQuery::Parameters & params) { MarkRanges ranges(1, MarkRange(0, part->size)); ExpressionBlockInputStream in(new MergeTreeBlockInputStream(full_path + part->name + '/', - DEFAULT_MERGE_BLOCK_SIZE, column_name, *this, part, ranges, StoragePtr(), false, NULL, ""), expr); + DEFAULT_MERGE_BLOCK_SIZE, column_name, *this, part, ranges, StoragePtr(), false, NULL, "", false), expr); MergedColumnOnlyOutputStream out(*this, full_path + part->name + '/', true); out.writePrefix(); From 3c4c400e685032a2a1c4f93ae2e9ae76335aef10 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Mar 2014 21:12:09 +0400 Subject: [PATCH 15/23] dbms: formatAST: better [#METR-2944]. --- dbms/src/Parsers/formatAST.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Parsers/formatAST.cpp b/dbms/src/Parsers/formatAST.cpp index 3fcfe94ae1e..cded21c44c3 100644 --- a/dbms/src/Parsers/formatAST.cpp +++ b/dbms/src/Parsers/formatAST.cpp @@ -224,11 +224,12 @@ void formatAST(const ASTSelectQuery & ast, std::ostream & s, size_t indent, bo void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { + std::string indent_str = one_line ? "" : std::string(4 * indent, ' '); std::string nl_or_nothing = one_line ? "" : "\n"; - s << nl_or_nothing << "(" << nl_or_nothing; + s << nl_or_nothing << indent_str << "(" << nl_or_nothing; formatAST(*ast.children[0], s, indent + 1, hilite, one_line); - s << nl_or_nothing << ")"; + s << nl_or_nothing << indent_str << ")"; } void formatAST(const ASTCreateQuery & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) From 34ef806287031068b3065490137378ba4cd4d482 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Mar 2014 21:15:14 +0400 Subject: [PATCH 16/23] dbms: fixed error [#METR-10377]. --- .../Interpreters/InterpreterSelectQuery.cpp | 24 +++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index de920019cbe..2250337710d 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -433,11 +433,31 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu Names required_columns = query_analyzer->getRequiredColumns(); if (table_function_storage) - table = table_function_storage; /// Если в запросе была указана табличная функция, данные читаем из нее. + { + /// Если в запросе была указана табличная функция, данные читаем из нее. + table = table_function_storage; + } else if (!query.table || !dynamic_cast(&*query.table)) + { + /// Запрос из обычной таблицы или без секции FROM. table = getTable(); + } else if (dynamic_cast(&*query.table)) - interpreter_subquery = new InterpreterSelectQuery(query.table, context, required_columns, QueryProcessingStage::Complete, subquery_depth + 1); + { + /** Для подзапроса не действуют ограничения на максимальный размер результата. + * Так как результат поздапроса - ещё не результат всего запроса. + */ + Context subquery_context = context; + Settings subquery_settings = context.getSettings(); + subquery_settings.limits.max_result_rows = 0; + subquery_settings.limits.max_result_bytes = 0; + /// Вычисление extremes не имеет смысла и не нужно (если его делать, то в результате всего запроса могут взяться extremes подзапроса). + subquery_settings.extremes = 0; + subquery_context.setSettings(subquery_settings); + + interpreter_subquery = new InterpreterSelectQuery( + query.table, subquery_context, required_columns, QueryProcessingStage::Complete, subquery_depth + 1); + } /// если в настройках установлен default_sample != 1, то все запросы выполняем с сэмплингом /// если таблица не поддерживает сэмплинг получим исключение From b8218d64b8088275f049ed75e9adb9c92be166ed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Mar 2014 22:20:03 +0400 Subject: [PATCH 17/23] dbms: Fixed accidentially introduced error [#METR-10377]. --- dbms/include/DB/Columns/ColumnSet.h | 3 +++ dbms/include/DB/Common/VirtualColumnUtils.h | 1 - dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 -- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/include/DB/Columns/ColumnSet.h b/dbms/include/DB/Columns/ColumnSet.h index a1abbb8af71..32346ac5da9 100644 --- a/dbms/include/DB/Columns/ColumnSet.h +++ b/dbms/include/DB/Columns/ColumnSet.h @@ -16,6 +16,9 @@ class ColumnSet : public IColumnDummy public: ColumnSet(size_t s_, SetPtr data_) : IColumnDummy(s_), data(data_) {} + /// Столбец не константный. Иначе столбец будет использоваться в вычислениях в ExpressionActions::prepare, когда множество из подзапроса ещё не готово. + bool isConst() const { return false; } + std::string getName() const { return "ColumnSet"; } ColumnPtr cloneDummy(size_t s_) const { return new ColumnSet(s_, data); } diff --git a/dbms/include/DB/Common/VirtualColumnUtils.h b/dbms/include/DB/Common/VirtualColumnUtils.h index 2e00b0d2b82..c6a2328eb61 100644 --- a/dbms/include/DB/Common/VirtualColumnUtils.h +++ b/dbms/include/DB/Common/VirtualColumnUtils.h @@ -10,7 +10,6 @@ #include #include #include -#include #include namespace DB diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 36567103bae..4d06a422031 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1229,9 +1229,7 @@ Sets ExpressionAnalyzer::getSetsWithSubqueries() { Sets res; for (auto & s : sets_with_subqueries) - { res.push_back(s.second); - } return res; } From 42dc0516d23e868fd28b2d52d218edffdeaafebe Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Thu, 13 Mar 2014 18:49:17 +0400 Subject: [PATCH 18/23] added config for zookeeper [#METR-10172] --- libs/libzkutil/include/zkutil/ZooKeeper.h | 5 +++ libs/libzkutil/src/ZooKeeper.cpp | 47 ++++++++++++++++++++++- 2 files changed, 50 insertions(+), 2 deletions(-) diff --git a/libs/libzkutil/include/zkutil/ZooKeeper.h b/libs/libzkutil/include/zkutil/ZooKeeper.h index 06c7900a5cd..ab875279aca 100644 --- a/libs/libzkutil/include/zkutil/ZooKeeper.h +++ b/libs/libzkutil/include/zkutil/ZooKeeper.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace zkutil @@ -19,6 +20,9 @@ class ZooKeeper public: ZooKeeper(const std::string & hosts, int32_t sessionTimeoutMs = DEFAULT_SESSION_TIMEOUT, WatchFunction * watch = nullptr); + ZooKeeper(const Poco::Util::LayeredConfiguration & config, const std::string & config_name, + WatchFunction * watch = nullptr); + /** Возвращает true, если сессия навсегда завершена. * Это возможно только если соединение было установлено, а потом разорвалось. Это достаточно редкая ситуация. * С другой стороны, если, например, указан неправильный сервер или порт, попытки соединения будут продолжаться бесконечно, @@ -95,6 +99,7 @@ public: OpResultsPtr tryMulti(const Ops & ops); private: + void init(const std::string & hosts, int32_t sessionTimeoutMs, WatchFunction * watch_); friend struct StateWatch; zk::ZooKeeper impl; diff --git a/libs/libzkutil/src/ZooKeeper.cpp b/libs/libzkutil/src/ZooKeeper.cpp index 190ba860881..c82c5f44e5b 100644 --- a/libs/libzkutil/src/ZooKeeper.cpp +++ b/libs/libzkutil/src/ZooKeeper.cpp @@ -43,9 +43,10 @@ struct StateWatch : public zk::Watch } }; -ZooKeeper::ZooKeeper(const std::string & hosts, int32_t sessionTimeoutMs, WatchFunction * watch_) - : state_watch(watch_) +void ZooKeeper::init(const std::string & hosts, int32_t sessionTimeoutMs, WatchFunction * watch_) { + state_watch = watch_; + CHECKED(impl.init(hosts, sessionTimeoutMs, boost::make_shared(this))); ACL perm; @@ -55,6 +56,48 @@ ZooKeeper::ZooKeeper(const std::string & hosts, int32_t sessionTimeoutMs, WatchF default_acl.push_back(perm); } +ZooKeeper::ZooKeeper(const std::string & hosts, int32_t sessionTimeoutMs, WatchFunction * watch_) +{ + init(hosts, sessionTimeoutMs, watch_); +} + +struct ZooKeeperArgs +{ + ZooKeeperArgs(const Poco::Util::LayeredConfiguration & config, const std::string & config_name) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_name, keys); + std::string node_key = "node"; + std::string node_key_ext = "node["; + + session_timeout_ms = DEFAULT_SESSION_TIMEOUT; + for (const auto & key : keys) + { + if (key == node_key || key.compare(0, node_key.size(), node_key) == 0) + { + if (hosts.size()) + hosts += std::string(" "); + hosts += config.getString(config_name + "." + key + ".host") + ":" + config.getString(config_name + "." + key + ".port"); + } + else if (key == "session_timeout_ms") + { + session_timeout_ms = config.getInt(config_name + "." + key); + } + else throw KeeperException(std::string("Unknown key ") + key + " in config file"); + } + } + + std::string hosts; + size_t session_timeout_ms; +}; + +ZooKeeper::ZooKeeper(const Poco::Util::LayeredConfiguration & config, const std::string & config_name, + WatchFunction * watch) +{ + ZooKeeperArgs args(config, config_name); + init(args.hosts, args.session_timeout_ms, watch); +} + void ZooKeeper::stateChanged(WatchEvent::type event, SessionState::type state, const std::string & path) { session_state = state; From 1e44f91f002621bde1e930386a7104f5eda4dd15 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 13 Mar 2014 21:30:58 +0400 Subject: [PATCH 19/23] dbms: fixed error in formatAST [#METR-10443]. --- dbms/src/Parsers/formatAST.cpp | 33 +++++++++++++++++++++++++++++---- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/dbms/src/Parsers/formatAST.cpp b/dbms/src/Parsers/formatAST.cpp index cded21c44c3..77d01059ec3 100644 --- a/dbms/src/Parsers/formatAST.cpp +++ b/dbms/src/Parsers/formatAST.cpp @@ -449,6 +449,10 @@ static void writeAlias(const String & name, std::ostream & s, bool hilite, bool void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { + /// Если есть алиас, то требуются скобки вокруг всего выражения, включая алиас. Потому что запись вида 0 AS x + 0 синтаксически некорректна. + if (need_parens && !ast.alias.empty()) + s << '('; + /// Стоит ли записать эту функцию в виде оператора? bool written = false; if (ast.arguments && !ast.parameters) @@ -558,30 +562,33 @@ void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent, bool written = true; } } + } + if (!written && ast.arguments->children.size() >= 1) + { if (!written && 0 == strcmp(ast.name.c_str(), "array")) { - s << '['; + s << (hilite ? hilite_operator : "") << '[' << (hilite ? hilite_none : ""); for (size_t i = 0; i < ast.arguments->children.size(); ++i) { if (i != 0) s << ", "; formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, false); } - s << ']'; + s << (hilite ? hilite_operator : "") << ']' << (hilite ? hilite_none : ""); written = true; } if (!written && 0 == strcmp(ast.name.c_str(), "tuple")) { - s << '('; + s << (hilite ? hilite_operator : "") << '(' << (hilite ? hilite_none : ""); for (size_t i = 0; i < ast.arguments->children.size(); ++i) { if (i != 0) s << ", "; formatAST(*ast.arguments->children[i], s, indent, hilite, one_line, false); } - s << ')'; + s << (hilite ? hilite_operator : "") << ')' << (hilite ? hilite_none : ""); written = true; } } @@ -609,11 +616,18 @@ void formatAST(const ASTFunction & ast, std::ostream & s, size_t indent, bool } if (!ast.alias.empty()) + { writeAlias(ast.alias, s, hilite, one_line); + if (need_parens) + s << ')'; + } } void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { + if (need_parens && !ast.alias.empty()) + s << '('; + s << (hilite ? hilite_identifier : ""); WriteBufferFromOStream wb(s, 32); @@ -623,15 +637,26 @@ void formatAST(const ASTIdentifier & ast, std::ostream & s, size_t indent, bo s << (hilite ? hilite_none : ""); if (!ast.alias.empty()) + { writeAlias(ast.alias, s, hilite, one_line); + if (need_parens) + s << ')'; + } } void formatAST(const ASTLiteral & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) { + if (need_parens && !ast.alias.empty()) + s << '('; + s << apply_visitor(FieldVisitorToString(), ast.value); if (!ast.alias.empty()) + { writeAlias(ast.alias, s, hilite, one_line); + if (need_parens) + s << ')'; + } } void formatAST(const ASTNameTypePair & ast, std::ostream & s, size_t indent, bool hilite, bool one_line, bool need_parens) From ccc3da3148b6c8c666f5691e6bba90ec77feb968 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Mar 2014 00:12:40 +0400 Subject: [PATCH 20/23] =?UTF-8?q?=E2=96=88=E2=96=88=E2=96=88=E2=96=88?= =?UTF-8?q?=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88?= =?UTF-8?q?:=20development=20[#METR-8766].?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbms/include/DB/IO/BufferWithOwnMemory.h | 25 +++++++++++++++++++----- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/dbms/include/DB/IO/BufferWithOwnMemory.h b/dbms/include/DB/IO/BufferWithOwnMemory.h index 32cc2e05498..0c044ddfb4c 100644 --- a/dbms/include/DB/IO/BufferWithOwnMemory.h +++ b/dbms/include/DB/IO/BufferWithOwnMemory.h @@ -20,12 +20,12 @@ namespace DB */ struct Memory : boost::noncopyable { - size_t m_capacity; - size_t m_size; - char * m_data; - size_t alignment; + size_t m_capacity = 0; + size_t m_size = 0; + char * m_data = nullptr; + size_t alignment = 0; - Memory() : m_capacity(0), m_size(0), m_data(NULL), alignment(0) {} + Memory() {} /// Если alignment != 0, то будет выделяться память, выровненная на alignment. Memory(size_t size_, size_t alignment_ = 0) : m_capacity(size_), m_size(m_capacity), alignment(alignment_) @@ -38,6 +38,21 @@ struct Memory : boost::noncopyable dealloc(); } + Memory(Memory && rhs) + { + *this = std::move(rhs); + } + + Memory & operator=(Memory && rhs) + { + std::swap(m_capacity, rhs.m_capacity); + std::swap(m_size, rhs.m_size); + std::swap(m_data, rhs.m_data); + std::swap(alignment, rhs.alignment); + + return *this; + } + size_t size() const { return m_size; } const char & operator[](size_t i) const { return m_data[i]; } char & operator[](size_t i) { return m_data[i]; } From 420d5696eaa7baede15fc6c554ba6295ce894017 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 14 Mar 2014 12:30:50 +0400 Subject: [PATCH 21/23] clickhouse: fixed ARRAY JOIN in distributed queries without WHERE and aggregation. [#METR-10349] --- .../DB/Interpreters/InterpreterSelectQuery.h | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 22 ++++++++++++++----- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/dbms/include/DB/Interpreters/InterpreterSelectQuery.h b/dbms/include/DB/Interpreters/InterpreterSelectQuery.h index 3b106749922..f932066e15c 100644 --- a/dbms/include/DB/Interpreters/InterpreterSelectQuery.h +++ b/dbms/include/DB/Interpreters/InterpreterSelectQuery.h @@ -65,7 +65,7 @@ private: void executeTotalsAndHaving( BlockInputStreams & streams, bool has_having, ExpressionActionsPtr expression, bool overflow_row); void executeHaving( BlockInputStreams & streams, ExpressionActionsPtr expression); - void executeOuterExpression( BlockInputStreams & streams, ExpressionActionsPtr expression); + void executeExpression( BlockInputStreams & streams, ExpressionActionsPtr expression); void executeOrder( BlockInputStreams & streams); void executePreLimit( BlockInputStreams & streams); void executeUnion( BlockInputStreams & streams); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 2250337710d..6dc8a892e85 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -209,7 +209,8 @@ BlockInputStreamPtr InterpreterSelectQuery::execute() bool need_aggregate = false; bool has_having = false; bool has_order_by = !query.order_expression_list.isNull(); - + + ExpressionActionsPtr array_join; ExpressionActionsPtr before_where; ExpressionActionsPtr before_aggregation; ExpressionActionsPtr before_having; @@ -225,8 +226,9 @@ BlockInputStreamPtr InterpreterSelectQuery::execute() if (from_stage < QueryProcessingStage::WithMergeableState && to_stage >= QueryProcessingStage::WithMergeableState) { - query_analyzer->appendArrayJoin(chain); - + if (query_analyzer->appendArrayJoin(chain)) + array_join = chain.getLastActions(); + if (query_analyzer->appendWhere(chain)) { has_where = true; @@ -312,6 +314,16 @@ BlockInputStreamPtr InterpreterSelectQuery::execute() if (need_aggregate) executeAggregation(streams, before_aggregation, aggregate_overflow_row, aggregate_final); + if (array_join && !has_where && !need_aggregate && to_stage == QueryProcessingStage::WithMergeableState) + { + /** Если есть ARRAY JOIN, его действие сначала старается оказаться в + * before_where, before_aggregation или before_order_and_select. + * Если ни одного из них нет, array_join нужно выполнить отдельно. + */ + + executeExpression(streams, array_join); + } + /** Оптимизация - при распределённой обработке запроса, * если не указаны DISTINCT, GROUP, HAVING, ORDER, но указан LIMIT, * то выполним предварительный LIMIT на удалёном сервере. @@ -339,7 +351,7 @@ BlockInputStreamPtr InterpreterSelectQuery::execute() executeHaving(streams, before_having); } - executeOuterExpression(streams, before_order_and_select); + executeExpression(streams, before_order_and_select); if (has_order_by) executeOrder(streams); @@ -667,7 +679,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(BlockInputStreams & streams, } -void InterpreterSelectQuery::executeOuterExpression(BlockInputStreams & streams, ExpressionActionsPtr expression) +void InterpreterSelectQuery::executeExpression(BlockInputStreams & streams, ExpressionActionsPtr expression) { bool is_async = settings.asynchronous && streams.size() <= settings.max_threads; for (BlockInputStreams::iterator it = streams.begin(); it != streams.end(); ++it) From 4384ba00097806bdefe60896941444116fa6b661 Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Thu, 13 Mar 2014 18:56:31 +0400 Subject: [PATCH 22/23] zookeeper: added config description [#METR-10172] --- libs/libzkutil/include/zkutil/ZooKeeper.h | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/libs/libzkutil/include/zkutil/ZooKeeper.h b/libs/libzkutil/include/zkutil/ZooKeeper.h index ab875279aca..09c8d01b552 100644 --- a/libs/libzkutil/include/zkutil/ZooKeeper.h +++ b/libs/libzkutil/include/zkutil/ZooKeeper.h @@ -20,6 +20,19 @@ class ZooKeeper public: ZooKeeper(const std::string & hosts, int32_t sessionTimeoutMs = DEFAULT_SESSION_TIMEOUT, WatchFunction * watch = nullptr); + /** конфиг вида + + + example1 + 2181 + + + example2 + 2181 + + 30000 + + */ ZooKeeper(const Poco::Util::LayeredConfiguration & config, const std::string & config_name, WatchFunction * watch = nullptr); From 806003d19ef7d8ddb0053d82fe1f1fc885bef4a4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Mar 2014 19:56:42 +0400 Subject: [PATCH 23/23] dbms: removed old commented code [#METR-2944]. --- dbms/include/DB/Columns/ColumnArray.h | 9 --------- 1 file changed, 9 deletions(-) diff --git a/dbms/include/DB/Columns/ColumnArray.h b/dbms/include/DB/Columns/ColumnArray.h index 165645d87f2..3b697510e22 100644 --- a/dbms/include/DB/Columns/ColumnArray.h +++ b/dbms/include/DB/Columns/ColumnArray.h @@ -452,41 +452,32 @@ private: for (size_t i = 0; i < col_size; ++i) { - // std::cerr << "i: " << i << std::endl; /// Насколько размножить массив. size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; - // std::cerr << "size_to_replicate: " << size_to_replicate << std::endl; /// Количество строк в массиве. size_t value_size = cur_offsets[i] - prev_cur_offset; - // std::cerr << "value_size: " << value_size << std::endl; size_t sum_chars_size = 0; for (size_t j = 0; j < size_to_replicate; ++j) { - // std::cerr << "j: " << j << std::endl; current_res_offset += value_size; res_offsets.push_back(current_res_offset); - // std::cerr << "current_res_offset: " << current_res_offset << std::endl; sum_chars_size = 0; size_t prev_cur_string_offset_local = prev_cur_string_offset; for (size_t k = 0; k < value_size; ++k) { - // std::cerr << "k: " << k << std::endl; /// Размер одной строки. size_t chars_size = cur_string_offsets[k + prev_cur_offset] - prev_cur_string_offset_local; - // std::cerr << "chars_size: " << chars_size << std::endl; current_res_string_offset += chars_size; res_string_offsets.push_back(current_res_string_offset); - // std::cerr << "current_res_string_offset: " << current_res_string_offset << std::endl; /// Копирование символов одной строки. res_chars.resize(res_chars.size() + chars_size); memcpy(&res_chars[res_chars.size() - chars_size], &cur_chars[prev_cur_string_offset_local], chars_size); - // std::cerr << "copied: " << mysqlxx::escape << std::string(reinterpret_cast(&cur_chars[prev_cur_string_offset_local]), chars_size) << std::endl; sum_chars_size += chars_size; prev_cur_string_offset_local += chars_size;